From 94d32611cac8418de3a36e21b4b553150d2523b0 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 3 Jun 2025 13:31:14 -0400 Subject: [PATCH 1/3] Make WindowedValue a public interface The following mostly-automated changes: - Moved WindowedValue from util to values package - Make WindowedValue an interface with companion class WindowedValues --- .../apache/beam/runners/core/DoFnRunner.java | 2 +- .../apache/beam/runners/core/DoFnRunners.java | 2 +- .../GroupAlsoByWindowViaWindowSetNewDoFn.java | 6 +- .../core/GroupByKeyViaGroupByKeyOnly.java | 4 +- .../beam/runners/core/KeyedWorkItem.java | 2 +- .../beam/runners/core/KeyedWorkItemCoder.java | 4 +- .../beam/runners/core/KeyedWorkItems.java | 2 +- .../core/LateDataDroppingDoFnRunner.java | 5 +- .../beam/runners/core/LateDataUtils.java | 2 +- ...oundedSplittableProcessElementInvoker.java | 2 +- .../beam/runners/core/ProcessFnRunner.java | 2 +- .../core/PushbackSideInputDoFnRunner.java | 2 +- .../beam/runners/core/ReduceFnRunner.java | 2 +- .../beam/runners/core/SideInputHandler.java | 2 +- .../beam/runners/core/SimpleDoFnRunner.java | 11 +- .../SimplePushbackSideInputDoFnRunner.java | 2 +- .../SplittableParDoViaKeyedWorkItems.java | 5 +- .../core/SplittableProcessElementInvoker.java | 2 +- .../beam/runners/core/StatefulDoFnRunner.java | 5 +- .../runners/core/KeyedWorkItemCoderTest.java | 9 +- .../core/LateDataDroppingDoFnRunnerTest.java | 5 +- ...edSplittableProcessElementInvokerTest.java | 4 +- .../beam/runners/core/ReduceFnRunnerTest.java | 2 +- .../beam/runners/core/ReduceFnTester.java | 7 +- .../runners/core/SideInputHandlerTest.java | 5 +- .../runners/core/SimpleDoFnRunnerTest.java | 36 +- ...SimplePushbackSideInputDoFnRunnerTest.java | 28 +- .../core/SplittableParDoProcessFnTest.java | 7 +- .../runners/core/StatefulDoFnRunnerTest.java | 45 +-- .../beam/runners/core/WindowMatchers.java | 2 +- .../beam/runners/core/WindowMatchersTest.java | 6 +- .../triggers/TriggerStateMachineTester.java | 6 +- .../direct/AbstractModelEnforcement.java | 2 +- .../direct/BoundedReadEvaluatorFactory.java | 7 +- .../runners/direct/CloningBundleFactory.java | 2 +- .../beam/runners/direct/CommittedBundle.java | 2 +- .../CreateViewNoopEvaluatorFactory.java | 2 +- .../direct/DirectTransformExecutor.java | 2 +- ...ycleManagerRemovingTransformEvaluator.java | 2 +- .../direct/FlattenEvaluatorFactory.java | 2 +- .../GroupAlsoByWindowEvaluatorFactory.java | 5 +- .../GroupByKeyOnlyEvaluatorFactory.java | 5 +- .../ImmutabilityCheckingBundleFactory.java | 2 +- .../ImmutabilityEnforcementFactory.java | 2 +- .../direct/ImmutableListBundleFactory.java | 2 +- .../direct/ImpulseEvaluatorFactory.java | 8 +- .../beam/runners/direct/ModelEnforcement.java | 2 +- .../beam/runners/direct/MultiStepCombine.java | 2 +- .../runners/direct/PCollectionViewWriter.java | 2 +- .../beam/runners/direct/ParDoEvaluator.java | 2 +- .../direct/ParDoMultiOverrideFactory.java | 7 +- .../direct/PassthroughTransformEvaluator.java | 2 +- .../beam/runners/direct/QuiescenceDriver.java | 4 +- .../runners/direct/SideInputContainer.java | 2 +- ...ttableProcessElementsEvaluatorFactory.java | 6 +- .../direct/StatefulParDoEvaluatorFactory.java | 2 +- .../runners/direct/StepTransformResult.java | 2 +- .../direct/TestStreamEvaluatorFactory.java | 9 +- .../runners/direct/TransformEvaluator.java | 2 +- .../beam/runners/direct/TransformResult.java | 2 +- .../direct/UnboundedReadEvaluatorFactory.java | 11 +- .../runners/direct/UncommittedBundle.java | 2 +- .../runners/direct/ViewEvaluatorFactory.java | 2 +- .../direct/WindowEvaluatorFactory.java | 5 +- .../BoundedReadEvaluatorFactoryTest.java | 11 +- .../direct/CloningBundleFactoryTest.java | 21 +- .../runners/direct/CommittedResultTest.java | 4 +- .../direct/DirectTransformExecutorTest.java | 19 +- ...ManagerRemovingTransformEvaluatorTest.java | 9 +- .../runners/direct/EvaluationContextTest.java | 11 +- .../direct/FlattenEvaluatorFactoryTest.java | 26 +- .../GroupByKeyEvaluatorFactoryTest.java | 27 +- .../GroupByKeyOnlyEvaluatorFactoryTest.java | 27 +- ...ImmutabilityCheckingBundleFactoryTest.java | 19 +- .../ImmutabilityEnforcementFactoryTest.java | 9 +- .../ImmutableListBundleFactoryTest.java | 31 +- .../direct/ImpulseEvaluatorFactoryTest.java | 5 +- .../KeyedPValueTrackingVisitorTest.java | 9 +- .../runners/direct/ParDoEvaluatorTest.java | 13 +- .../direct/SideInputContainerTest.java | 31 +- .../StatefulParDoEvaluatorFactoryTest.java | 5 +- .../TestStreamEvaluatorFactoryTest.java | 15 +- .../UnboundedReadEvaluatorFactoryTest.java | 13 +- .../direct/ViewEvaluatorFactoryTest.java | 7 +- .../runners/direct/WatermarkManagerTest.java | 32 +- .../direct/WindowEvaluatorFactoryTest.java | 9 +- .../FlinkBatchPortablePipelineTranslator.java | 13 +- .../flink/FlinkBatchTransformTranslators.java | 16 +- .../flink/FlinkBatchTranslationContext.java | 7 +- ...FlinkStreamingAggregationsTranslators.java | 15 +- ...nkStreamingPortablePipelineTranslator.java | 39 +- .../FlinkStreamingTransformTranslators.java | 25 +- .../FlinkStreamingTranslationContext.java | 5 +- .../adapter/BeamFlinkDataSetAdapter.java | 7 +- .../adapter/BeamFlinkDataStreamAdapter.java | 7 +- .../metrics/DoFnRunnerWithMetricsUpdate.java | 2 +- .../functions/AbstractFlinkCombineRunner.java | 2 +- .../functions/FlinkAssignContext.java | 2 +- .../functions/FlinkAssignWindows.java | 5 +- .../functions/FlinkDoFnFunction.java | 7 +- .../FlinkExecutableStageFunction.java | 2 +- .../FlinkExecutableStagePruningFunction.java | 2 +- .../FlinkExplodeWindowsFunction.java | 2 +- .../FlinkMergingNonShuffleReduceFunction.java | 2 +- .../FlinkMultiOutputPruningFunction.java | 2 +- .../FlinkNonMergingReduceFunction.java | 5 +- .../functions/FlinkPartialReduceFunction.java | 2 +- .../functions/FlinkReduceFunction.java | 2 +- .../functions/FlinkStatefulDoFnFunction.java | 2 +- .../functions/HashingFlinkCombineRunner.java | 5 +- .../functions/ImpulseSourceFunction.java | 5 +- .../functions/SideInputInitializer.java | 2 +- .../SingleWindowFlinkCombineRunner.java | 5 +- .../functions/SortingFlinkCombineRunner.java | 11 +- .../translation/types/KvKeySelector.java | 2 +- .../types/WindowedKvKeySelector.java | 2 +- .../wrappers/ImpulseInputFormat.java | 5 +- .../wrappers/SourceInputFormat.java | 5 +- .../wrappers/streaming/DoFnOperator.java | 2 +- .../ExecutableStageDoFnOperator.java | 11 +- .../streaming/KvToFlinkKeyKeySelector.java | 2 +- .../PartialReduceBundleOperator.java | 5 +- .../streaming/SdfFlinkKeyKeySelector.java | 2 +- .../streaming/SingletonKeyedWorkItem.java | 2 +- .../SingletonKeyedWorkItemCoder.java | 7 +- .../streaming/SplittableDoFnOperator.java | 9 +- .../streaming/WindowDoFnOperator.java | 5 +- .../streaming/WorkItemKeySelector.java | 2 +- .../streaming/io/DedupingOperator.java | 2 +- .../streaming/io/StreamingImpulseSource.java | 5 +- .../streaming/io/TestStreamSource.java | 5 +- .../streaming/io/UnboundedSourceWrapper.java | 5 +- .../io/source/bounded/FlinkBoundedSource.java | 2 +- .../bounded/FlinkBoundedSourceReader.java | 5 +- .../unbounded/FlinkUnboundedSource.java | 2 +- .../unbounded/FlinkUnboundedSourceReader.java | 5 +- .../stableinput/BufferedElements.java | 2 +- .../stableinput/BufferingDoFnRunner.java | 2 +- .../flink/FlinkPipelineOptionsTest.java | 9 +- .../streaming/BoundedSourceRestoreTest.java | 2 +- .../FlinkExecutableStageFunctionTest.java | 15 +- .../functions/ImpulseSourceFunctionTest.java | 2 +- .../streaming/DedupingOperatorTest.java | 18 +- .../wrappers/streaming/DoFnOperatorTest.java | 342 +++++++++--------- .../ExecutableStageDoFnOperatorTest.java | 45 +-- .../streaming/StreamRecordStripper.java | 2 +- .../streaming/WindowDoFnOperatorTest.java | 19 +- .../io/UnboundedSourceWrapperTest.java | 2 +- .../bounded/FlinkBoundedSourceReaderTest.java | 2 +- .../FlinkUnboundedSourceReaderTest.java | 2 +- .../stableinput/BufferedElementsTest.java | 8 +- .../stableinput/BufferingDoFnRunnerTest.java | 4 +- .../dataflow/BatchStatefulParDoOverrides.java | 7 +- .../runners/dataflow/BatchViewOverrides.java | 18 +- .../dataflow/DataflowPipelineTranslator.java | 6 +- .../beam/runners/dataflow/DataflowRunner.java | 12 +- .../dataflow/util/CloudObjectTranslators.java | 2 +- .../runners/dataflow/util/CloudObjects.java | 2 +- .../dataflow/BatchViewOverridesTest.java | 84 ++--- .../dataflow/util/CloudObjectsTest.java | 4 +- .../worker/AssignWindowsParDoFnFactory.java | 5 +- .../worker/BatchModeUngroupingParDoFn.java | 2 +- ...reateIsmShardKeyAndSortKeyDoFnFactory.java | 2 +- .../worker/DataflowOutputCounter.java | 2 +- .../DataflowPortabilityPCollectionView.java | 2 +- .../worker/DataflowProcessFnRunner.java | 5 +- .../worker/GroupAlsoByWindowFnRunner.java | 5 +- .../GroupAlsoByWindowParDoFnFactory.java | 2 +- .../worker/GroupAlsoByWindowsParDoFn.java | 2 +- .../worker/GroupingShuffleReader.java | 4 +- .../worker/GroupingShuffleReaderFactory.java | 2 +- ...uffleReaderWithFaultyBytesReadCounter.java | 2 +- .../IntrinsicMapTaskExecutorFactory.java | 2 +- .../runners/dataflow/worker/IsmReader.java | 2 +- .../dataflow/worker/IsmReaderFactory.java | 4 +- .../dataflow/worker/IsmReaderImpl.java | 2 +- .../dataflow/worker/IsmSideInputReader.java | 2 +- .../beam/runners/dataflow/worker/IsmSink.java | 2 +- .../dataflow/worker/IsmSinkFactory.java | 4 +- .../PairWithConstantKeyDoFnFactory.java | 2 +- .../worker/PartialGroupByKeyParDoFns.java | 13 +- .../worker/PartitioningShuffleReader.java | 4 +- .../PartitioningShuffleReaderFactory.java | 2 +- .../dataflow/worker/PubsubDynamicSink.java | 2 +- .../runners/dataflow/worker/PubsubReader.java | 7 +- .../runners/dataflow/worker/PubsubSink.java | 4 +- ...eifyTimestampAndWindowsParDoFnFactory.java | 7 +- .../runners/dataflow/worker/ShuffleSink.java | 4 +- .../dataflow/worker/ShuffleSinkFactory.java | 2 +- .../dataflow/worker/SimpleParDoFn.java | 2 +- .../worker/SplittableProcessFnFactory.java | 6 +- ...StreamingGroupAlsoByWindowReshuffleFn.java | 2 +- ...amingKeyedWorkItemSideInputDoFnRunner.java | 2 +- ...amingPCollectionViewWriterDoFnFactory.java | 2 +- ...StreamingPCollectionViewWriterParDoFn.java | 2 +- .../worker/StreamingSideInputDoFnRunner.java | 2 +- .../worker/StreamingSideInputFetcher.java | 5 +- .../ToIsmRecordForMultimapDoFnFactory.java | 2 +- .../worker/UngroupedWindmillReader.java | 9 +- .../dataflow/worker/ValuesDoFnFactory.java | 2 +- .../worker/WindmillKeyedWorkItem.java | 5 +- .../worker/WindmillReaderIteratorBase.java | 2 +- .../runners/dataflow/worker/WindmillSink.java | 4 +- .../worker/WindowingWindmillReader.java | 4 +- .../dataflow/worker/WorkerCustomSources.java | 7 +- .../BatchGroupAlsoByWindowAndCombineFn.java | 2 +- .../worker/util/BatchGroupAlsoByWindowFn.java | 2 +- .../BatchGroupAlsoByWindowReshuffleFn.java | 2 +- .../BatchGroupAlsoByWindowViaIteratorsFn.java | 2 +- ...tchGroupAlsoByWindowViaOutputBufferFn.java | 2 +- .../worker/util/ValueInEmptyWindows.java | 9 +- .../ComputationWorkExecutorFactory.java | 8 +- .../worker/AvroByteReaderFactoryTest.java | 6 +- .../worker/AvroByteSinkFactoryTest.java | 4 +- .../worker/CombineValuesFnFactoryTest.java | 42 +-- ...eIsmShardKeyAndSortKeyDoFnFactoryTest.java | 2 +- ...ataflowPortabilityPCollectionViewTest.java | 2 +- .../worker/DefaultParDoFnFactoryTest.java | 4 +- .../worker/GroupingShuffleReaderTest.java | 21 +- .../IntrinsicMapTaskExecutorFactoryTest.java | 6 +- .../dataflow/worker/IsmReaderFactoryTest.java | 8 +- .../dataflow/worker/IsmReaderTest.java | 2 +- .../worker/IsmSideInputReaderTest.java | 53 +-- .../runners/dataflow/worker/IsmSinkTest.java | 2 +- .../PairWithConstantKeyDoFnFactoryTest.java | 2 +- .../worker/PartialGroupByKeyParDoFnsTest.java | 75 ++-- .../worker/PartitioningShuffleReaderTest.java | 25 +- .../worker/PubsubDynamicSinkTest.java | 11 +- .../dataflow/worker/PubsubReaderTest.java | 11 +- .../dataflow/worker/PubsubSinkTest.java | 20 +- .../dataflow/worker/ReaderFactoryTest.java | 5 +- .../dataflow/worker/ReaderTestUtils.java | 2 +- ...TimestampAndWindowsParDoFnFactoryTest.java | 7 +- .../worker/ShuffleReaderFactoryTest.java | 4 +- .../worker/ShuffleSinkFactoryTest.java | 14 +- .../dataflow/worker/ShuffleSinkTest.java | 15 +- .../dataflow/worker/SimpleParDoFnTest.java | 48 +-- .../worker/StreamingDataflowWorkerTest.java | 19 +- .../StreamingGroupAlsoByWindowFnsTest.java | 2 +- ...ngGroupAlsoByWindowsReshuffleDoFnTest.java | 2 +- ...gKeyedWorkItemSideInputDoFnRunnerTest.java | 5 +- ...gPCollectionViewWriterDoFnFactoryTest.java | 4 +- .../StreamingSideInputDoFnRunnerTest.java | 7 +- .../worker/StreamingSideInputFetcherTest.java | 5 +- ...ToIsmRecordForMultimapDoFnFactoryTest.java | 2 +- .../worker/UngroupedShuffleReaderTest.java | 7 +- .../worker/UserParDoFnFactoryTest.java | 18 +- .../worker/ValuesDoFnFactoryTest.java | 2 +- .../worker/WindmillKeyedWorkItemTest.java | 8 +- .../WindmillReaderIteratorBaseTest.java | 5 +- .../worker/WorkerCustomSourcesTest.java | 4 +- .../graph/LengthPrefixUnknownCodersTest.java | 15 +- ...tchGroupAlsoByWindowReshuffleDoFnTest.java | 9 +- ...pAlsoByWindowsViaOutputBufferDoFnTest.java | 2 +- .../util/GroupAlsoByWindowProperties.java | 72 ++-- .../worker/util/ListOutputManager.java | 2 +- .../control/BundleCheckpointHandlers.java | 5 +- .../control/ProcessBundleDescriptors.java | 4 +- .../BatchSideInputHandlerFactory.java | 2 +- .../translation/PipelineTranslatorUtils.java | 9 +- .../runners/fnexecution/wire/WireCoders.java | 7 +- .../control/RemoteExecutionTest.java | 11 +- .../control/SdkHarnessClientTest.java | 19 +- .../fnexecution/data/GrpcDataServiceTest.java | 23 +- .../BatchSideInputHandlerFactoryTest.java | 30 +- .../fnexecution/wire/CommonCoderTest.java | 12 +- .../wire/LengthPrefixUnknownCodersTest.java | 14 +- .../apache/beam/runners/jet/DAGBuilder.java | 7 +- .../runners/jet/JetTransformTranslators.java | 5 +- .../runners/jet/JetTranslationContext.java | 8 +- .../org/apache/beam/runners/jet/Utils.java | 18 +- .../jet/processors/AbstractParDoP.java | 5 +- .../runners/jet/processors/AssignWindowP.java | 5 +- .../jet/processors/BoundedSourceP.java | 5 +- .../beam/runners/jet/processors/FlattenP.java | 2 +- .../beam/runners/jet/processors/ImpulseP.java | 4 +- .../jet/processors/StatefulParDoP.java | 2 +- .../jet/processors/UnboundedSourceP.java | 4 +- .../beam/runners/jet/processors/ViewP.java | 7 +- .../runners/jet/processors/WindowGroupP.java | 11 +- .../apache/beam/runners/jet/TestStreamP.java | 4 +- .../org/apache/beam/runners/local/Bundle.java | 2 +- .../samza/adapter/BoundedSourceSystem.java | 5 +- .../samza/adapter/UnboundedSourceSystem.java | 5 +- .../samza/metrics/DoFnRunnerWithMetrics.java | 2 +- .../samza/metrics/SamzaGBKMetricOp.java | 2 +- .../runners/samza/metrics/SamzaMetricOp.java | 2 +- .../samza/runtime/AsyncDoFnRunner.java | 2 +- .../samza/runtime/ClassicBundleManager.java | 2 +- .../beam/runners/samza/runtime/DoFnOp.java | 5 +- .../runtime/DoFnRunnerWithKeyedInternals.java | 2 +- .../samza/runtime/FutureCollector.java | 2 +- .../samza/runtime/FutureCollectorImpl.java | 2 +- .../runners/samza/runtime/GroupByKeyOp.java | 5 +- .../samza/runtime/KvToKeyedWorkItemOp.java | 2 +- .../apache/beam/runners/samza/runtime/Op.java | 2 +- .../beam/runners/samza/runtime/OpAdapter.java | 2 +- .../beam/runners/samza/runtime/OpEmitter.java | 2 +- .../beam/runners/samza/runtime/OpMessage.java | 2 +- .../runners/samza/runtime/PortableDoFnOp.java | 2 +- .../samza/runtime/SamzaAssignContext.java | 2 +- .../samza/runtime/SamzaDoFnRunners.java | 2 +- .../samza/runtime/SingletonKeyedWorkItem.java | 2 +- ...SplittableParDoProcessKeyedElementsOp.java | 7 +- .../runners/samza/runtime/WindowAssignOp.java | 5 +- .../translation/GroupByKeyTranslator.java | 9 +- .../ParDoBoundMultiTranslator.java | 17 +- .../samza/translation/ReadTranslator.java | 2 +- .../translation/ReshuffleTranslator.java | 5 +- .../SamzaImpulseSystemFactory.java | 4 +- .../SamzaPublishViewTranslator.java | 2 +- .../SamzaTestStreamSystemFactory.java | 5 +- .../SplittableParDoTranslators.java | 2 +- .../samza/translation/TranslationContext.java | 5 +- .../beam/runners/samza/util/SamzaCoders.java | 5 +- .../util/SamzaPipelineTranslatorUtils.java | 6 +- .../beam/runners/samza/util/WindowUtils.java | 8 +- .../samza/adapter/TestSourceHelpers.java | 4 +- .../TestSamzaRunnerWithTransformMetrics.java | 11 +- .../samza/runtime/AsyncDoFnRunnerTest.java | 7 +- .../runtime/ClassicBundleManagerTest.java | 2 +- .../runtime/FutureCollectorImplTest.java | 2 +- .../io/BoundedDatasetFactory.java | 4 +- .../translation/EvaluationContext.java | 2 +- .../translation/PipelineTranslator.java | 2 +- .../translation/SparkSessionFactory.java | 8 +- .../translation/TransformTranslator.java | 2 +- .../translation/batch/Aggregators.java | 5 +- .../batch/CombineGloballyTranslatorBatch.java | 5 +- .../CombineGroupedValuesTranslatorBatch.java | 2 +- .../batch/CombinePerKeyTranslatorBatch.java | 5 +- .../batch/DoFnPartitionIteratorFactory.java | 2 +- .../translation/batch/DoFnRunnerFactory.java | 2 +- .../batch/DoFnRunnerWithMetrics.java | 2 +- .../batch/FlattenTranslatorBatch.java | 2 +- .../translation/batch/GroupByKeyHelpers.java | 5 +- .../batch/GroupByKeyTranslatorBatch.java | 5 +- .../batch/ImpulseTranslatorBatch.java | 5 +- .../batch/ParDoTranslatorBatch.java | 2 +- .../batch/ReadSourceTranslatorBatch.java | 2 +- .../batch/ReshuffleTranslatorBatch.java | 2 +- .../batch/WindowAssignTranslatorBatch.java | 5 +- .../GroupAlsoByWindowViaOutputBufferFn.java | 5 +- .../batch/functions/SideInputValues.java | 4 +- .../translation/helpers/EncoderHelpers.java | 5 +- .../translation/batch/AggregatorsTest.java | 7 +- .../batch/functions/SideInputValuesTest.java | 9 +- .../helpers/EncoderHelpersTest.java | 5 +- .../coders/SparkRunnerKryoRegistrator.java | 3 +- .../beam/runners/spark/io/SourceRDD.java | 5 +- .../spark/io/SparkUnboundedSource.java | 7 +- .../SparkGroupAlsoByWindowViaWindowSet.java | 7 +- .../spark/stateful/StateSpecFunctions.java | 9 +- .../spark/translation/BoundedDataset.java | 17 +- .../translation/DoFnRunnerWithMetrics.java | 2 +- .../spark/translation/EvaluationContext.java | 5 +- .../translation/GroupCombineFunctions.java | 4 +- .../GroupNonMergingWindowsFunctions.java | 15 +- .../spark/translation/MultiDoFnFunction.java | 2 +- .../ReifyTimestampsAndWindowsFunction.java | 5 +- .../spark/translation/SideInputMetadata.java | 2 +- .../translation/SparkAssignWindowFn.java | 5 +- .../SparkBatchPortablePipelineTranslator.java | 15 +- .../spark/translation/SparkCombineFn.java | 18 +- ...parkExecutableStageExtractionFunction.java | 2 +- .../SparkExecutableStageFunction.java | 4 +- ...arkGroupAlsoByWindowViaOutputBufferFn.java | 5 +- .../translation/SparkInputDataProcessor.java | 2 +- .../translation/SparkPCollectionView.java | 2 +- ...rkStreamingPortablePipelineTranslator.java | 19 +- .../translation/TransformTranslator.java | 27 +- .../spark/translation/TranslationUtils.java | 7 +- .../streaming/ParDoStateUpdateFn.java | 11 +- .../StatefulStreamingParDoEvaluator.java | 11 +- .../StreamingTransformTranslator.java | 21 +- .../translation/streaming/TestDStream.java | 7 +- .../streaming/UnboundedDataset.java | 2 +- .../streaming/WatermarkSyncedDStream.java | 2 +- .../spark/util/SparkSideInputReader.java | 2 +- .../GroupNonMergingWindowsFunctionsTest.java | 13 +- .../spark/translation/SparkCombineFnTest.java | 7 +- .../SparkExecutableStageFunctionTest.java | 21 +- .../SparkInputDataProcessorTest.java | 13 +- .../translation/TransformTranslatorTest.java | 25 +- .../ValueAndCoderLazySerializableTest.java | 13 +- .../Twister2BatchTranslationContext.java | 2 +- .../twister2/Twister2TranslationContext.java | 2 +- .../wrappers/Twister2BoundedSource.java | 5 +- .../wrappers/Twister2EmptySource.java | 2 +- .../batch/AssignWindowTranslatorBatch.java | 2 +- .../batch/FlattenTranslatorBatch.java | 2 +- .../batch/GroupByKeyTranslatorBatch.java | 7 +- .../batch/ImpulseTranslatorBatch.java | 2 +- .../batch/PCollectionViewTranslatorBatch.java | 11 +- .../ParDoMultiOutputTranslatorBatch.java | 2 +- .../batch/ReadSourceTranslatorBatch.java | 2 +- .../functions/AssignWindowsFunction.java | 5 +- .../functions/ByteToElemFunction.java | 4 +- .../functions/ByteToWindowFunction.java | 4 +- .../ByteToWindowFunctionPrimitive.java | 9 +- .../translators/functions/DoFnFunction.java | 2 +- .../functions/ElemToBytesFunction.java | 9 +- .../functions/GroupByWindowFunction.java | 5 +- .../translators/functions/ImpulseSource.java | 5 +- .../functions/MapToTupleFunction.java | 13 +- .../functions/OutputTagFilter.java | 2 +- .../twister2/utils/TranslationUtils.java | 5 +- .../twister2/utils/Twister2AssignContext.java | 2 +- .../utils/Twister2SideInputReader.java | 2 +- .../apache/beam/sdk/transforms/Create.java | 7 +- .../util/construction/CoderTranslators.java | 21 +- .../construction/ModelCoderRegistrar.java | 12 +- .../construction/RunnerPCollectionView.java | 2 +- .../apache/beam/sdk/values/WindowedValue.java | 60 +++ .../WindowedValues.java} | 168 ++++----- .../data/BeamFnDataInboundObserverTest.java | 17 +- .../beam/sdk/transforms/CreateTest.java | 9 +- .../beam/sdk/transforms/RedistributeTest.java | 10 +- .../beam/sdk/transforms/ReshuffleTest.java | 10 +- .../beam/sdk/util/WindowedValueTest.java | 43 ++- .../construction/CoderTranslationTest.java | 6 +- .../util/construction/ModelCodersTest.java | 2 +- .../euphoria/core/testkit/WindowingTest.java | 1 + .../jmh/PrecombineGroupingTableBenchmark.java | 11 +- .../harness/jmh/ProcessBundleBenchmark.java | 4 +- .../beam/fn/harness/AssignWindowsRunner.java | 5 +- .../beam/fn/harness/BeamFnDataReadRunner.java | 2 +- .../fn/harness/BeamFnDataWriteRunner.java | 2 +- .../beam/fn/harness/CombineRunners.java | 4 +- .../apache/beam/fn/harness/FlattenRunner.java | 2 +- .../beam/fn/harness/FnApiDoFnRunner.java | 83 ++--- .../apache/beam/fn/harness/MapFnRunners.java | 2 +- .../fn/harness/PTransformRunnerFactory.java | 2 +- .../fn/harness/PrecombineGroupingTable.java | 7 +- ...littablePairWithRestrictionDoFnRunner.java | 5 +- ...bleSplitAndSizeRestrictionsDoFnRunner.java | 5 +- .../beam/fn/harness/WindowedSplitResult.java | 2 +- .../harness/control/ProcessBundleHandler.java | 2 +- .../data/PCollectionConsumerRegistry.java | 7 +- .../beam/fn/harness/debug/ElementSample.java | 2 +- .../beam/fn/harness/debug/OutputSampler.java | 5 +- .../fn/harness/state/FnApiStateAccessor.java | 6 +- .../fn/harness/AssignWindowsRunnerTest.java | 35 +- .../fn/harness/BeamFnDataReadRunnerTest.java | 7 +- .../fn/harness/BeamFnDataWriteRunnerTest.java | 7 +- .../beam/fn/harness/CombineRunnersTest.java | 4 +- .../beam/fn/harness/FlattenRunnerTest.java | 9 +- .../beam/fn/harness/FnApiDoFnRunnerTest.java | 95 ++--- .../beam/fn/harness/MapFnRunnersTest.java | 13 +- .../PTransformRunnerFactoryTestContext.java | 2 +- .../harness/PrecombineGroupingTableTest.java | 6 +- ...ablePairWithRestrictionDoFnRunnerTest.java | 19 +- ...plitAndSizeRestrictionsDoFnRunnerTest.java | 33 +- .../data/BeamFnDataGrpcClientTest.java | 7 +- .../data/PCollectionConsumerRegistryTest.java | 4 +- .../fn/harness/debug/DataSamplerTest.java | 5 +- .../fn/harness/debug/OutputSamplerTest.java | 41 ++- .../components/deadletterqueue/DLQRouter.java | 1 + .../deadletterqueue/sinks/ThrowingSink.java | 2 +- .../deadletterqueue/DLQRouterTest.java | 1 + .../io/gcp/bigquery/BigQueryHelpersTest.java | 4 +- .../sdk/io/iceberg/RecordWriterManager.java | 2 +- .../io/iceberg/WriteGroupedRowsToFiles.java | 5 +- .../io/iceberg/WriteUngroupedRowsToFiles.java | 5 +- .../io/iceberg/RecordWriterManagerTest.java | 9 +- 465 files changed, 2211 insertions(+), 1925 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValue.java rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util/WindowedValue.java => values/WindowedValues.java} (88%) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java index 7c5c80fa4340..dce4fcaa68bd 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java @@ -20,7 +20,7 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java index a02b11d5c630..abe8b99b1a26 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java @@ -29,10 +29,10 @@ import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetNewDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetNewDoFn.java index 3e42bb54494e..9e2f4f4efce1 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetNewDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetNewDoFn.java @@ -25,10 +25,10 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.SystemDoFnInternal; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; @@ -99,7 +99,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - outputManager.output(mainTag, WindowedValue.of(output, timestamp, windows, pane)); + outputManager.output(mainTag, WindowedValues.of(output, timestamp, windows, pane)); } @Override @@ -109,7 +109,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - outputManager.output(tag, WindowedValue.of(output, timestamp, windows, pane)); + outputManager.output(tag, WindowedValues.of(output, timestamp, windows, pane)); } }; } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java index 803128e24d73..ab4dbabd5d4e 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java @@ -29,10 +29,10 @@ import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItem.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItem.java index e035998c38dc..4901c5cbed5b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItem.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItem.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.core; import org.apache.beam.runners.core.TimerInternals.TimerData; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** * Interface that contains all the timers and elements associated with a specific work item. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java index 756a89c88aca..657e8d85638a 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java @@ -28,8 +28,8 @@ import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.StructuredCoder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** A {@link Coder} for {@link KeyedWorkItem KeyedWorkItems}. */ diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItems.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItems.java index 4c6df0c21051..db2ef3595505 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItems.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItems.java @@ -20,7 +20,7 @@ import java.util.Collections; import java.util.Objects; import org.apache.beam.runners.core.TimerInternals.TimerData; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java index ca1a0393b95d..0fb8ff52d1e1 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java @@ -25,8 +25,9 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowTracing; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.joda.time.Instant; @@ -140,7 +141,7 @@ public Iterable> filter( timerInternals.currentOutputWatermarkTime()); } else { nonLateElements.add( - WindowedValue.of( + WindowedValues.of( element.getValue(), element.getTimestamp(), window, element.getPane())); } } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataUtils.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataUtils.java index d6f237bf571c..fbb7b315c3b1 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataUtils.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataUtils.java @@ -21,7 +21,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.WindowTracing; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.FluentIterable; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java index 51cd8c690aee..b7ba99e32f55 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java @@ -45,11 +45,11 @@ import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimator; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Futures; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ProcessFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ProcessFnRunner.java index 28bdf45abd92..b8a4dcf7cb7f 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ProcessFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ProcessFnRunner.java @@ -27,9 +27,9 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.joda.time.Instant; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java index 1b2d90eb6181..61feaffad919 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java @@ -20,7 +20,7 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java index b9eef50ed59f..ead53515558f 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java @@ -47,9 +47,9 @@ import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowTracing; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java index 8b2cb5c3db2d..31afffda44ea 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java @@ -37,8 +37,8 @@ import org.apache.beam.sdk.transforms.Materializations.MultimapView; import org.apache.beam.sdk.transforms.ViewFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index b375d38c5a98..1b9c4640b85e 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -54,10 +54,11 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.FluentIterable; @@ -331,7 +332,7 @@ public void output(OutputT output, Instant timestamp, BoundedWindow window) { @Override public void output(TupleTag tag, T output, Instant timestamp, BoundedWindow window) { - outputWindowedValue(tag, WindowedValue.of(output, timestamp, window, PaneInfo.NO_FIRING)); + outputWindowedValue(tag, WindowedValues.of(output, timestamp, window, PaneInfo.NO_FIRING)); } } @@ -447,7 +448,7 @@ public void outputWindowedValue( Collection windows, PaneInfo paneInfo) { SimpleDoFnRunner.this.outputWindowedValue( - tag, WindowedValue.of(output, timestamp, windows, paneInfo)); + tag, WindowedValues.of(output, timestamp, windows, paneInfo)); } @Override @@ -888,7 +889,7 @@ public void outputWindowedValue( PaneInfo paneInfo) { checkTimestamp(timestamp(), timestamp); SimpleDoFnRunner.this.outputWindowedValue( - tag, WindowedValue.of(output, timestamp, windows, paneInfo)); + tag, WindowedValues.of(output, timestamp, windows, paneInfo)); } @Override @@ -1117,7 +1118,7 @@ public void outputWindowedValue( PaneInfo paneInfo) { checkTimestamp(this.timestamp, timestamp); SimpleDoFnRunner.this.outputWindowedValue( - tag, WindowedValue.of(output, timestamp, windows, paneInfo)); + tag, WindowedValues.of(output, timestamp, windows, paneInfo)); } @Override diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java index fb04b536b4c0..2be8071f9835 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java @@ -24,8 +24,8 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java index caa5565541a4..772428f84b65 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java @@ -45,7 +45,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformReplacements; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.PTransformTranslation.RawPTransform; @@ -59,6 +58,8 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @@ -295,7 +296,7 @@ public ProcessFn( this.elementTag = StateTags.value( "element", - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( elementCoder, inputWindowingStrategy.getWindowFn().windowCoder())); this.restrictionTag = StateTags.value("restriction", restrictionCoder); this.watermarkEstimatorStateTag = diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableProcessElementInvoker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableProcessElementInvoker.java index 00a72b9197bb..1ff66d6e517c 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableProcessElementInvoker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableProcessElementInvoker.java @@ -25,8 +25,8 @@ import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimator; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java index 0b4e980b7f5f..52bbb3a306a0 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java @@ -38,7 +38,8 @@ import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowTracing; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.joda.time.Duration; @@ -98,7 +99,7 @@ public StatefulDoFnRunner( this.sortBufferTag = StateTags.makeSystemTagInternal( - StateTags.bag(SORT_BUFFER_STATE, WindowedValue.getFullCoder(inputCoder, windowCoder))); + StateTags.bag(SORT_BUFFER_STATE, WindowedValues.getFullCoder(inputCoder, windowCoder))); rejectMergingWindowFn(windowFn); } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/KeyedWorkItemCoderTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/KeyedWorkItemCoderTest.java index 65c9d099d62b..6099f1fdc8e1 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/KeyedWorkItemCoderTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/KeyedWorkItemCoderTest.java @@ -23,7 +23,8 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Instant; import org.junit.Test; @@ -50,9 +51,9 @@ public void testEncodeDecodeEqual() throws Exception { TimeDomain.EVENT_TIME)); Iterable> elements = ImmutableList.of( - WindowedValue.valueInGlobalWindow(1), - WindowedValue.valueInGlobalWindow(4), - WindowedValue.valueInGlobalWindow(8)); + WindowedValues.valueInGlobalWindow(1), + WindowedValues.valueInGlobalWindow(4), + WindowedValues.valueInGlobalWindow(8)); KeyedWorkItemCoder coder = KeyedWorkItemCoder.of(StringUtf8Coder.of(), VarIntCoder.of(), GlobalWindow.Coder.INSTANCE); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java index 89cfb4df2486..5a6257ba0a1b 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java @@ -29,7 +29,8 @@ import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -97,7 +98,7 @@ public void testLateDataFilter() throws Exception { private WindowedValue createDatum(T element, long timestampMillis) { Instant timestamp = new Instant(timestampMillis); - return WindowedValue.of( + return WindowedValues.of( element, timestamp, Arrays.asList(WINDOW_FN.assignWindow(timestamp)), PaneInfo.NO_FIRING); } } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvokerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvokerTest.java index 14a52128f1a9..13a6709a43fd 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvokerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvokerTest.java @@ -41,8 +41,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles; import org.joda.time.Duration; import org.joda.time.Instant; @@ -139,7 +139,7 @@ public void outputWindowedValue( SplittableProcessElementInvoker.Result rval = invoker.invokeProcessElement( DoFnInvokers.invokerFor(fn), - WindowedValue.of(null, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), + WindowedValues.of(null, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), new OffsetRangeTracker(initialRestriction), new WatermarkEstimator() { @Override diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java index ea8bb406b60d..048bfb069451 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java @@ -78,9 +78,9 @@ import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java index d047a9133930..193ff51f2051 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java @@ -57,11 +57,12 @@ import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowTracing; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Equivalence; @@ -552,7 +553,7 @@ public final void injectElements(List> values) throws E windowFn.assignWindows( new TestAssignContext( windowFn, value, timestamp, GlobalWindow.INSTANCE)); - return WindowedValue.of(value, timestamp, windows, PaneInfo.NO_FIRING); + return WindowedValues.of(value, timestamp, windows, PaneInfo.NO_FIRING); } catch (Exception e) { throw new RuntimeException(e); } @@ -610,7 +611,7 @@ public void outputWindowedValue( KV copy = SerializableUtils.ensureSerializableByCoder( KvCoder.of(StringUtf8Coder.of(), outputCoder), output, "outputForWindow"); - WindowedValue> value = WindowedValue.of(copy, timestamp, windows, pane); + WindowedValue> value = WindowedValues.of(copy, timestamp, windows, pane); outputs.add(value); } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java index b76d8fe6b823..293a7fe72c7e 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java @@ -32,9 +32,10 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Duration; import org.joda.time.Instant; @@ -249,6 +250,6 @@ public void testMultipleSideInputs() { @SuppressWarnings({"unchecked", "rawtypes"}) private WindowedValue> valuesInWindow( List values, Instant timestamp, BoundedWindow window) { - return (WindowedValue) WindowedValue.of(values, timestamp, window, PaneInfo.NO_FIRING); + return (WindowedValue) WindowedValues.of(values, timestamp, window, PaneInfo.NO_FIRING); } } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java index a50eb7d9647e..a0435fe3026d 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java @@ -46,9 +46,10 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ArrayListMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ListMultimap; @@ -108,7 +109,7 @@ public void testProcessElementExceptionsWrappedAsUserCodeException() { thrown.expect(UserCodeException.class); thrown.expectCause(is(fn.exceptionToThrow)); - runner.processElement(WindowedValue.valueInGlobalWindow("anyValue")); + runner.processElement(WindowedValues.valueInGlobalWindow("anyValue")); } @Test @@ -170,7 +171,7 @@ public void testTimerSet() { Instant currentTime = new Instant(42); when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(currentTime); - runner.processElement(WindowedValue.valueInGlobalWindow("anyValue")); + runner.processElement(WindowedValues.valueInGlobalWindow("anyValue")); verify(mockTimerInternals) .setTimer( @@ -303,14 +304,14 @@ public void testBackwardsInTimeNoSkew() { runner.startBundle(); // An element output at the current timestamp is fine. runner.processElement( - WindowedValue.timestampedValueInGlobalWindow(Duration.ZERO, new Instant(0))); + WindowedValues.timestampedValueInGlobalWindow(Duration.ZERO, new Instant(0))); Exception exception = assertThrows( UserCodeException.class, () -> { // An element output before (current time - skew) is forbidden runner.processElement( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( Duration.millis(1L), new Instant(0))); }); @@ -354,7 +355,8 @@ public void testSkew() { runner.startBundle(); // Outputting between "now" and "now - allowed skew" succeeds. runner.processElement( - WindowedValue.timestampedValueInGlobalWindow(Duration.standardMinutes(5L), new Instant(0))); + WindowedValues.timestampedValueInGlobalWindow( + Duration.standardMinutes(5L), new Instant(0))); Exception exception = assertThrows( @@ -362,7 +364,7 @@ public void testSkew() { () -> { // Outputting before "now - allowed skew" fails. runner.processElement( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( Duration.standardHours(1L), new Instant(0))); }); @@ -405,12 +407,12 @@ public void testInfiniteSkew() { runner.startBundle(); runner.processElement( - WindowedValue.timestampedValueInGlobalWindow(Duration.millis(1L), new Instant(0))); + WindowedValues.timestampedValueInGlobalWindow(Duration.millis(1L), new Instant(0))); runner.processElement( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( Duration.millis(1L), BoundedWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)))); runner.processElement( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( // This is the maximum amount a timestamp in beam can move (from the maximum timestamp // to the minimum timestamp). Duration.millis(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) @@ -444,7 +446,7 @@ public void testTimerBackwardsInTimeNoSkew() { runner.startBundle(); // A timer with output timestamp at the current timestamp is fine. runner.processElement( - WindowedValue.timestampedValueInGlobalWindow(KV.of("1", Duration.ZERO), new Instant(0))); + WindowedValues.timestampedValueInGlobalWindow(KV.of("1", Duration.ZERO), new Instant(0))); Exception exception = assertThrows( @@ -452,7 +454,7 @@ public void testTimerBackwardsInTimeNoSkew() { () -> { // A timer with output timestamp before (current time - skew) is forbidden runner.processElement( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of("2", Duration.millis(1L)), new Instant(0))); }); @@ -495,7 +497,7 @@ public void testTimerSkew() { runner.startBundle(); // Timer with output timestamp between "now" and "now - allowed skew" succeeds. runner.processElement( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of("1", Duration.standardMinutes(5L)), new Instant(0))); Exception exception = @@ -504,7 +506,7 @@ public void testTimerSkew() { () -> { // A timer with output timestamp before (current time - skew) is forbidden runner.processElement( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of("2", Duration.standardHours(1L)), new Instant(0))); }); @@ -547,14 +549,14 @@ public void testTimerInfiniteSkew() { runner.startBundle(); runner.processElement( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of("1", Duration.millis(1L)), new Instant(0))); runner.processElement( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of("2", Duration.millis(1L)), BoundedWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)))); runner.processElement( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of( "3", // This is the maximum amount a timestamp in beam can move (from the maximum diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java index 7887faccbb06..b7dda599b8d2 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java @@ -56,10 +56,11 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.IdentitySideInputWindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -181,7 +182,7 @@ public void processElementSideInputNotReady() { createRunner(ImmutableList.of(singletonView)); WindowedValue oneWindow = - WindowedValue.of( + WindowedValues.of( 2, new Instant(-2), new IntervalWindow(new Instant(-500L), new Instant(0L)), @@ -201,7 +202,7 @@ public void processElementSideInputNotReadyMultipleWindows() { createRunner(ImmutableList.of(singletonView)); WindowedValue multiWindow = - WindowedValue.of( + WindowedValues.of( 2, new Instant(-2), ImmutableList.of( @@ -231,7 +232,7 @@ public void processElementSideInputNotReadySomeWindows() { IntervalWindow bigWindow = new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)); WindowedValue multiWindow = - WindowedValue.of( + WindowedValues.of( 2, new Instant(-2), ImmutableList.of(littleWindow, bigWindow, GlobalWindow.INSTANCE), @@ -240,13 +241,14 @@ public void processElementSideInputNotReadySomeWindows() { runner.processElementInReadyWindows(multiWindow); assertThat( multiWindowPushback, - containsInAnyOrder(WindowedValue.timestampedValueInGlobalWindow(2, new Instant(-2L)))); + containsInAnyOrder(WindowedValues.timestampedValueInGlobalWindow(2, new Instant(-2L)))); assertThat( underlying.inputElems, containsInAnyOrder( - WindowedValue.of( + WindowedValues.of( 2, new Instant(-2), ImmutableList.of(littleWindow), PaneInfo.NO_FIRING), - WindowedValue.of(2, new Instant(-2), ImmutableList.of(bigWindow), PaneInfo.NO_FIRING))); + WindowedValues.of( + 2, new Instant(-2), ImmutableList.of(bigWindow), PaneInfo.NO_FIRING))); } @Test @@ -258,7 +260,7 @@ public void processElementSideInputReadyAllWindows() { SimplePushbackSideInputDoFnRunner runner = createRunner(views); WindowedValue multiWindow = - WindowedValue.of( + WindowedValues.of( 2, new Instant(-2), ImmutableList.of( @@ -279,7 +281,7 @@ public void processElementNoSideInputs() { SimplePushbackSideInputDoFnRunner runner = createRunner(ImmutableList.of()); WindowedValue multiWindow = - WindowedValue.of( + WindowedValues.of( 2, new Instant(-2), ImmutableList.of( @@ -396,7 +398,7 @@ public void testLateDroppingForStatefulDoFnRunner() throws Exception { .thenReturn(true); WindowedValue multiWindow = - WindowedValue.of( + WindowedValues.of( 1, new Instant(0), ImmutableList.of(new IntervalWindow(new Instant(0), new Instant(0L + WINDOW_SIZE))), @@ -433,20 +435,20 @@ public void testGarbageCollectForStatefulDoFnRunner() throws Exception { // first element, key is hello, WINDOW_1 runner.processElementInReadyWindows( - WindowedValue.of(KV.of("hello", 1), elementTime, WINDOW_1, PaneInfo.NO_FIRING)); + WindowedValues.of(KV.of("hello", 1), elementTime, WINDOW_1, PaneInfo.NO_FIRING)); assertEquals(1, (int) stateInternals.state(windowNamespace(WINDOW_1), stateTag).read()); // second element, key is hello, WINDOW_2 runner.processElementInReadyWindows( - WindowedValue.of( + WindowedValues.of( KV.of("hello", 1), elementTime.plus(Duration.millis(WINDOW_SIZE)), WINDOW_2, PaneInfo.NO_FIRING)); runner.processElementInReadyWindows( - WindowedValue.of( + WindowedValues.of( KV.of("hello", 1), elementTime.plus(Duration.millis(WINDOW_SIZE)), WINDOW_2, diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoProcessFnTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoProcessFnTest.java index c6fe4e9bc671..7820fd36a835 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoProcessFnTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoProcessFnTest.java @@ -61,12 +61,13 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.ValueInSingleWindow; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; @@ -208,7 +209,7 @@ public void close() throws Exception { /** Performs a seed {@link DoFn.ProcessElement} call feeding the element and restriction. */ void startElement(InputT element, RestrictionT restriction) throws Exception { startElement( - WindowedValue.of( + WindowedValues.of( KV.of(element, restriction), currentProcessingTime, GlobalWindow.INSTANCE, @@ -324,7 +325,7 @@ public void testTrivialProcessFnPropagatesOutputWindowAndTimestamp() throws Exce MAX_OUTPUTS_PER_BUNDLE, MAX_BUNDLE_DURATION); tester.startElement( - WindowedValue.of( + WindowedValues.of( KV.of(42, new SomeRestriction()), base, Collections.singletonList(w), diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java index b65e6af84d96..11f3d244b37f 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java @@ -44,9 +44,10 @@ import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.checkerframework.checker.nullness.qual.Nullable; @@ -152,7 +153,7 @@ public void testDataDroppedBasedOnInputWatermarkWhenOrdered() throws Exception { new IntervalWindow(timestamp, timestamp.plus(Duration.millis(WINDOW_SIZE))); runner.processElement( - WindowedValue.of(KV.of("hello", 1), timestamp, window, PaneInfo.NO_FIRING)); + WindowedValues.of(KV.of("hello", 1), timestamp, window, PaneInfo.NO_FIRING)); long droppedValues = container @@ -165,7 +166,7 @@ public void testDataDroppedBasedOnInputWatermarkWhenOrdered() throws Exception { timerInternals.advanceInputWatermark(timestamp.plus(Duration.millis(ALLOWED_LATENESS + 1))); runner.processElement( - WindowedValue.of(KV.of("hello", 1), timestamp, window, PaneInfo.NO_FIRING)); + WindowedValues.of(KV.of("hello", 1), timestamp, window, PaneInfo.NO_FIRING)); droppedValues = container @@ -195,7 +196,7 @@ private void testLateDropping(boolean ordered) throws Exception { Instant timestamp = new Instant(0); runner.processElement( - WindowedValue.of(KV.of("hello", 1), timestamp, window, PaneInfo.NO_FIRING)); + WindowedValues.of(KV.of("hello", 1), timestamp, window, PaneInfo.NO_FIRING)); long droppedValues = container @@ -220,7 +221,7 @@ private void testGarbageCollect(boolean ordered) throws Exception { // first element, key is hello, WINDOW_1 runner.processElement( - WindowedValue.of(KV.of("hello", 1), elementTime, WINDOW_1, PaneInfo.NO_FIRING)); + WindowedValues.of(KV.of("hello", 1), elementTime, WINDOW_1, PaneInfo.NO_FIRING)); if (ordered) { // move forward in time so that the input might get flushed @@ -232,14 +233,14 @@ private void testGarbageCollect(boolean ordered) throws Exception { // second element, key is hello, WINDOW_2 runner.processElement( - WindowedValue.of( + WindowedValues.of( KV.of("hello", 1), elementTime.plus(Duration.millis(WINDOW_SIZE)), WINDOW_2, PaneInfo.NO_FIRING)); runner.processElement( - WindowedValue.of( + WindowedValues.of( KV.of("hello", 1), elementTime.plus(Duration.millis(WINDOW_SIZE)), WINDOW_2, @@ -306,9 +307,9 @@ private void testOutput( // write two elements, with descending timestamps runner.processElement( - WindowedValue.of(KV.of("hello", 1), elementTime, WINDOW_1, PaneInfo.NO_FIRING)); + WindowedValues.of(KV.of("hello", 1), elementTime, WINDOW_1, PaneInfo.NO_FIRING)); runner.processElement( - WindowedValue.of( + WindowedValues.of( KV.of("hello", 2), elementTime.minus(Duration.millis(1)), WINDOW_1, @@ -327,17 +328,17 @@ private void testOutput( Arrays.asList( KV.of( outputTag, - WindowedValue.of( + WindowedValues.of( 2, elementTime.minus(Duration.millis(1)), WINDOW_1, PaneInfo.NO_FIRING)), - KV.of(outputTag, WindowedValue.of(3, elementTime, WINDOW_1, PaneInfo.NO_FIRING))), + KV.of(outputTag, WindowedValues.of(3, elementTime, WINDOW_1, PaneInfo.NO_FIRING))), outputs); } else { assertEquals( Arrays.asList( - KV.of(outputTag, WindowedValue.of(1, elementTime, WINDOW_1, PaneInfo.NO_FIRING)), + KV.of(outputTag, WindowedValues.of(1, elementTime, WINDOW_1, PaneInfo.NO_FIRING)), KV.of( outputTag, - WindowedValue.of( + WindowedValues.of( 3, elementTime.minus(Duration.millis(1)), WINDOW_1, PaneInfo.NO_FIRING))), outputs); } @@ -346,17 +347,17 @@ private void testOutput( // another window elementTime = elementTime.plus(Duration.millis(WINDOW_SIZE)); runner.processElement( - WindowedValue.of(KV.of("hello", 1), elementTime, WINDOW_2, PaneInfo.NO_FIRING)); + WindowedValues.of(KV.of("hello", 1), elementTime, WINDOW_2, PaneInfo.NO_FIRING)); runner.processElement( - WindowedValue.of( + WindowedValues.of( KV.of("hello", 2), elementTime.minus(Duration.millis(1)), WINDOW_2, PaneInfo.NO_FIRING)); runner.processElement( - WindowedValue.of( + WindowedValues.of( KV.of("hello", 3), elementTime.minus(Duration.millis(2)), WINDOW_2, @@ -375,25 +376,25 @@ private void testOutput( Arrays.asList( KV.of( outputTag, - WindowedValue.of( + WindowedValues.of( 3, elementTime.minus(Duration.millis(2)), WINDOW_2, PaneInfo.NO_FIRING)), KV.of( outputTag, - WindowedValue.of( + WindowedValues.of( 5, elementTime.minus(Duration.millis(1)), WINDOW_2, PaneInfo.NO_FIRING)), - KV.of(outputTag, WindowedValue.of(6, elementTime, WINDOW_2, PaneInfo.NO_FIRING))), + KV.of(outputTag, WindowedValues.of(6, elementTime, WINDOW_2, PaneInfo.NO_FIRING))), outputs); } else { assertEquals( Arrays.asList( - KV.of(outputTag, WindowedValue.of(1, elementTime, WINDOW_2, PaneInfo.NO_FIRING)), + KV.of(outputTag, WindowedValues.of(1, elementTime, WINDOW_2, PaneInfo.NO_FIRING)), KV.of( outputTag, - WindowedValue.of( + WindowedValues.of( 3, elementTime.minus(Duration.millis(1)), WINDOW_2, PaneInfo.NO_FIRING)), KV.of( outputTag, - WindowedValue.of( + WindowedValues.of( 6, elementTime.minus(Duration.millis(2)), WINDOW_2, PaneInfo.NO_FIRING))), outputs); } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java index ee33eb28f69d..0cdb4042471b 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java @@ -22,7 +22,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.hamcrest.Description; import org.hamcrest.Matcher; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchersTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchersTest.java index a5f36be1134b..9dd8ac502fde 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchersTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchersTest.java @@ -21,7 +21,7 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Instant; import org.junit.Test; @@ -39,7 +39,7 @@ public void testIsWindowedValueExact() { long windowEnd = 200; assertThat( - WindowedValue.of( + WindowedValues.of( "hello", new Instant(timestamp), new IntervalWindow(new Instant(windowStart), new Instant(windowEnd)), @@ -60,7 +60,7 @@ public void testIsWindowedValueReorderedWindows() { long windowEnd2 = 150; assertThat( - WindowedValue.of( + WindowedValues.of( "hello", new Instant(timestamp), ImmutableList.of( diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java index 096f3ac608b6..21dfddce7a6e 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java @@ -48,8 +48,9 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowTracing; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; import org.checkerframework.checker.nullness.qual.Nullable; @@ -248,7 +249,8 @@ public final void injectElements(Collection> values) th activeWindows.addActiveForTesting(window); } - windowedValues.add(WindowedValue.of(value, timestamp, assignedWindows, PaneInfo.NO_FIRING)); + windowedValues.add( + WindowedValues.of(value, timestamp, assignedWindows, PaneInfo.NO_FIRING)); } catch (Exception e) { throw new RuntimeException(e); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AbstractModelEnforcement.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AbstractModelEnforcement.java index 40faf5a0fe2a..bb7c299abf85 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AbstractModelEnforcement.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AbstractModelEnforcement.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** * An abstract {@link ModelEnforcement} that provides default empty implementations for each method. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java index bbb1ae842444..b31798e7e46b 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java @@ -33,11 +33,12 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.ReadTranslation; import org.apache.beam.sdk.util.construction.SplittableParDo.PrimitiveBoundedRead; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -152,7 +153,7 @@ public void processElement(WindowedValue> element) UncommittedBundle output = evaluationContext.createBundle(outputPCollection); while (contentsRemaining) { output.add( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( reader.getCurrent(), reader.getCurrentTimestamp())); contentsRemaining = reader.advance(); } @@ -221,7 +222,7 @@ public Collection>> getInitialInputs( CommittedBundle> inputShard = evaluationContext .>createRootBundle() - .add(WindowedValue.valueInGlobalWindow(BoundedSourceShard.of(bundle))) + .add(WindowedValues.valueInGlobalWindow(BoundedSourceShard.of(bundle))) .commit(BoundedWindow.TIMESTAMP_MAX_VALUE); shards.add(inputShard); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningBundleFactory.java index 7295b9227d83..c3d30bfbf744 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningBundleFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningBundleFactory.java @@ -22,8 +22,8 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.joda.time.Instant; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedBundle.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedBundle.java index 72546fc83f99..936429442638 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedBundle.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedBundle.java @@ -20,8 +20,8 @@ import org.apache.beam.runners.local.Bundle; import org.apache.beam.runners.local.StructuralKey; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CreateViewNoopEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CreateViewNoopEvaluatorFactory.java index fe1389cb89cd..d85890d217f7 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CreateViewNoopEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CreateViewNoopEvaluatorFactory.java @@ -20,7 +20,7 @@ import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.View; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.checkerframework.checker.nullness.qual.Nullable; /** diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTransformExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTransformExecutor.java index ee71e33ff93f..fb07a3fa919f 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTransformExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTransformExecutor.java @@ -27,8 +27,8 @@ import org.apache.beam.runners.core.metrics.MetricsContainerImpl; import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java index dbc52d82727e..6508cb2a78b2 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java @@ -19,7 +19,7 @@ import org.apache.beam.runners.core.TimerInternals.TimerData; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java index 92a65fe289e3..e1167d47a035 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java @@ -21,9 +21,9 @@ import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.Flatten.PCollections; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; /** diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java index c12a2ff44f8e..45cb58b4a492 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java @@ -44,11 +44,12 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowTracing; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -254,7 +255,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - bundle.add(WindowedValue.of(output, timestamp, windows, pane)); + bundle.add(WindowedValues.of(output, timestamp, windows, pane)); } @Override diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java index 3371d477fe2e..121c6f8bd5d9 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java @@ -33,9 +33,10 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; /** @@ -136,7 +137,7 @@ public TransformResult> finishBundle() { StructuralKey.of(key, keyCoder), (PCollection>) Iterables.getOnlyElement(application.getOutputs().values())); - bundle.add(WindowedValue.valueInGlobalWindow(groupedKv)); + bundle.add(WindowedValues.valueInGlobalWindow(groupedKv)); resultBuilder.addOutput(bundle); } return resultBuilder.build(); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java index 3faaa986bee3..370829451d36 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java @@ -27,8 +27,8 @@ import org.apache.beam.sdk.util.IllegalMutationException; import org.apache.beam.sdk.util.MutationDetector; import org.apache.beam.sdk.util.MutationDetectors; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.SetMultimap; import org.joda.time.Instant; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java index bc7fc2867e71..84509bded752 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java @@ -29,7 +29,7 @@ import org.apache.beam.sdk.util.MutationDetector; import org.apache.beam.sdk.util.MutationDetectors; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** * {@link ModelEnforcement} that enforces elements are not modified over the course of processing an diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java index b601f41a6171..5cfb6fa59374 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java @@ -25,8 +25,8 @@ import javax.annotation.Nonnull; import org.apache.beam.runners.local.StructuralKey; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImpulseEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImpulseEvaluatorFactory.java index 09337f3728b4..09eb1cfd8747 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImpulseEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImpulseEvaluatorFactory.java @@ -23,9 +23,10 @@ import org.apache.beam.sdk.transforms.Impulse; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.checkerframework.checker.nullness.qual.Nullable; @@ -67,7 +68,8 @@ public void processElement(WindowedValue element) throws Exception PCollection outputPCollection = (PCollection) Iterables.getOnlyElement(transform.getOutputs().values()); result.addOutput( - ctxt.createBundle(outputPCollection).add(WindowedValue.valueInGlobalWindow(new byte[0]))); + ctxt.createBundle(outputPCollection) + .add(WindowedValues.valueInGlobalWindow(new byte[0]))); } @Override @@ -94,7 +96,7 @@ public Collection> getInitialInputs( int targetParallelism) { return Collections.singleton( ctxt.createRootBundle() - .add(WindowedValue.valueInGlobalWindow(new ImpulseShard())) + .add(WindowedValues.valueInGlobalWindow(new ImpulseShard())) .commit(BoundedWindow.TIMESTAMP_MIN_VALUE)); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java index 335727e6c8d0..de8ffa3b9b4d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java @@ -19,8 +19,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; /** * Enforcement tools that verify that executing code conforms to the model. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java index 9a7b7f7a9b5d..e1573abf7864 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java @@ -46,13 +46,13 @@ import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.PTransformTranslation.RawPTransform; import org.apache.beam.sdk.util.construction.SingleInputOutputOverrideFactory; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.checkerframework.checker.nullness.qual.EnsuresNonNull; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PCollectionViewWriter.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PCollectionViewWriter.java index f3828d8a49dd..c37fcec735de 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PCollectionViewWriter.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PCollectionViewWriter.java @@ -17,9 +17,9 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; /** * A {@link PCollectionViewWriter} is responsible for writing contents of a {@link PCollection} to a diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java index 8370f95a419e..6be73b645c68 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java @@ -42,10 +42,10 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java index dd9ef1cc3984..3a5237fb19e4 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java @@ -43,7 +43,6 @@ import org.apache.beam.sdk.transforms.windowing.Repeatedly; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformReplacements; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.util.construction.ReplacementOutputs; @@ -56,6 +55,8 @@ import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; @@ -190,7 +191,7 @@ PCollection>> groupToKeyedWorkItem( return input // Stash the original timestamps, etc, for when it is fed to the user's DoFn .apply("Reify timestamps", ParDo.of(new ReifyWindowedValueFn<>())) - .setCoder(KvCoder.of(keyCoder, WindowedValue.getFullCoder(kvCoder, windowCoder))) + .setCoder(KvCoder.of(keyCoder, WindowedValues.getFullCoder(kvCoder, windowCoder))) // We are going to GBK to gather keys and windows but otherwise do not want // to alter the flow of data. This entails: @@ -317,7 +318,7 @@ public void processElement(final ProcessContext c, final BoundedWindow window) { c.output( KV.of( c.element().getKey(), - WindowedValue.of(c.element(), c.timestamp(), window, c.pane()))); + WindowedValues.of(c.element(), c.timestamp(), window, c.pane()))); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java index bad15989a57b..530d5021ddb1 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.direct; import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; class PassthroughTransformEvaluator implements TransformEvaluator { public static PassthroughTransformEvaluator create( diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/QuiescenceDriver.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/QuiescenceDriver.java index 13cf2ebfe844..8915c25bef87 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/QuiescenceDriver.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/QuiescenceDriver.java @@ -34,9 +34,9 @@ import org.apache.beam.runners.local.ExecutionDriver; import org.apache.beam.runners.local.PipelineMessageReceiver; import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; import org.slf4j.Logger; @@ -196,7 +196,7 @@ private void fireTimers() { (PCollection) Iterables.getOnlyElement( transformTimers.getExecutable().getMainInputs().values())) - .add(WindowedValue.valueInGlobalWindow(work)) + .add(WindowedValues.valueInGlobalWindow(work)) .commit(evaluationContext.now()); processBundle( bundle, diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java index 62b0a06ddff1..3fe6cf4a11b8 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java @@ -38,9 +38,9 @@ import org.apache.beam.sdk.transforms.ViewFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java index a5bf31e406d2..373490f69c44 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java @@ -36,11 +36,11 @@ import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheLoader; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.MoreExecutors; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -155,7 +155,7 @@ public void outputWindowedValue( Collection windows, PaneInfo pane) { outputManager.output( - transform.getMainOutputTag(), WindowedValue.of(output, timestamp, windows, pane)); + transform.getMainOutputTag(), WindowedValues.of(output, timestamp, windows, pane)); } @Override @@ -165,7 +165,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - outputManager.output(tag, WindowedValue.of(output, timestamp, windows, pane)); + outputManager.output(tag, WindowedValues.of(output, timestamp, windows, pane)); } }; SideInputReader sideInputReader = diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java index 9d8a815ede6b..8edbd45f2b5d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java @@ -37,10 +37,10 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheLoader; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.joda.time.Instant; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java index ae4da1ec0717..8cfe07bdbb71 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java @@ -31,7 +31,7 @@ import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Instant; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java index 85d0371a451e..8ca2a909e367 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java @@ -33,7 +33,6 @@ import org.apache.beam.sdk.testing.TestStream.WatermarkEvent; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.ReplacementOutputs; import org.apache.beam.sdk.util.construction.TestStreamTranslation; import org.apache.beam.sdk.values.PBegin; @@ -41,6 +40,8 @@ import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; @@ -110,7 +111,7 @@ public void processElement(WindowedValue> element) throws Exc (PCollection) Iterables.getOnlyElement(application.getOutputs().values())); for (TimestampedValue elem : ((ElementEvent) event).getElements()) { bundle.add( - WindowedValue.timestampedValueInGlobalWindow(elem.getValue(), elem.getTimestamp())); + WindowedValues.timestampedValueInGlobalWindow(elem.getValue(), elem.getTimestamp())); } resultBuilder.addOutput(bundle); } @@ -127,7 +128,7 @@ public void processElement(WindowedValue> element) throws Exc TestStreamIndex next = streamIndex.next(); if (next.getIndex() < events.size()) { resultBuilder.addUnprocessedElements( - Collections.singleton(WindowedValue.timestampedValueInGlobalWindow(next, watermark))); + Collections.singleton(WindowedValues.timestampedValueInGlobalWindow(next, watermark))); } } @@ -231,7 +232,7 @@ public Collection>> getInitialInputs( CommittedBundle> initialBundle = evaluationContext .>createRootBundle() - .add(WindowedValue.valueInGlobalWindow(TestStreamIndex.of(testStream.original))) + .add(WindowedValues.valueInGlobalWindow(TestStreamIndex.of(testStream.original))) .commit(BoundedWindow.TIMESTAMP_MAX_VALUE); return Collections.singleton(initialBundle); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluator.java index 5d3a6a784f00..3eb10e0dce34 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluator.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** * An evaluator of a specific application of a transform. Will be used for at least one {@link diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java index 7e1a8f61c4b3..2817ddb97510 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java @@ -27,7 +27,7 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java index bfcb0a591824..e45990a760ea 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java @@ -35,11 +35,12 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.ReadTranslation; import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; @@ -143,7 +144,7 @@ public void processElement( do { if (deduplicator.shouldOutput(reader.getCurrentRecordId())) { output.add( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( reader.getCurrent(), reader.getCurrentTimestamp())); } numElements++; @@ -168,7 +169,7 @@ public void processElement( .addOutput(output) .addUnprocessedElements( Collections.singleton( - WindowedValue.timestampedValueInGlobalWindow(residual, watermark))); + WindowedValues.timestampedValueInGlobalWindow(residual, watermark))); } else { Instant watermark = reader.getWatermark(); if (watermark.isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE)) { @@ -176,7 +177,7 @@ public void processElement( // Might be better to finalize old checkpoint. resultBuilder.addUnprocessedElements( Collections.>singleton( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( UnboundedSourceShard.of( shard.getSource(), shard.getDeduplicator(), @@ -344,7 +345,7 @@ static class InputProvider initialShards.add( evaluationContext .>createRootBundle() - .add(WindowedValue.valueInGlobalWindow(shard)) + .add(WindowedValues.valueInGlobalWindow(shard)) .commit(BoundedWindow.TIMESTAMP_MAX_VALUE)); } return initialShards.build(); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundle.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundle.java index 1885703d7121..83c64dc3bbd2 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundle.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundle.java @@ -18,8 +18,8 @@ package org.apache.beam.runners.direct; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java index d5ea6f06b95b..823462494bd7 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java @@ -22,8 +22,8 @@ import org.apache.beam.runners.direct.CommittedResult.OutputType; import org.apache.beam.runners.direct.DirectWriteViewVisitor.WriteView; import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; /** diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java index 1f5292b4e7c9..17139a7495c3 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java @@ -23,9 +23,10 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.WindowIntoTranslation; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; @@ -90,7 +91,7 @@ public void processElement(WindowedValue compressedElement) throws Excep for (WindowedValue element : compressedElement.explodeWindows()) { Collection windows = assignWindows(windowFn, element); outputBundle.add( - WindowedValue.of( + WindowedValues.of( element.getValue(), element.getTimestamp(), windows, element.getPane())); } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java index 121c4325427a..12f8eae152fc 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java @@ -53,9 +53,10 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.hamcrest.Matchers; @@ -262,7 +263,7 @@ public void boundedSourceInMemoryTransformEvaluatorShardsOfSource() throws Excep UncommittedBundle> rootBundle = bundleFactory.createRootBundle(); for (BoundedSource split : splits) { BoundedSourceShard shard = BoundedSourceShard.of(split); - rootBundle.add(WindowedValue.valueInGlobalWindow(shard)); + rootBundle.add(WindowedValues.valueInGlobalWindow(shard)); } CommittedBundle> shards = rootBundle.commit(Instant.now()); @@ -301,7 +302,7 @@ public void boundedSourceEvaluatorClosesReader() throws Exception { TransformEvaluator> evaluator = factory.forApplication( sourceTransform, bundleFactory.createRootBundle().commit(Instant.now())); - evaluator.processElement(WindowedValue.valueInGlobalWindow(BoundedSourceShard.of(source))); + evaluator.processElement(WindowedValues.valueInGlobalWindow(BoundedSourceShard.of(source))); evaluator.finishBundle(); CommittedBundle committed = output.commit(Instant.now()); assertThat(committed.getElements(), containsInAnyOrder(gw(2L), gw(3L), gw(1L))); @@ -321,7 +322,7 @@ public void boundedSourceEvaluatorNoElementsClosesReader() throws Exception { TransformEvaluator> evaluator = factory.forApplication( sourceTransform, bundleFactory.createRootBundle().commit(Instant.now())); - evaluator.processElement(WindowedValue.valueInGlobalWindow(BoundedSourceShard.of(source))); + evaluator.processElement(WindowedValues.valueInGlobalWindow(BoundedSourceShard.of(source))); evaluator.finishBundle(); CommittedBundle committed = output.commit(Instant.now()); assertThat(committed.getElements(), emptyIterable()); @@ -453,6 +454,6 @@ public void close() throws IOException { } private static WindowedValue gw(Long elem) { - return WindowedValue.valueInGlobalWindow(elem); + return WindowedValues.valueInGlobalWindow(elem); } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java index e03f453e21ad..6d98e3c32acc 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java @@ -39,9 +39,10 @@ import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.WithKeys; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.joda.time.Instant; @@ -61,8 +62,8 @@ public class CloningBundleFactoryTest { @Test public void rootBundleSucceedsIgnoresCoder() { - WindowedValue one = WindowedValue.valueInGlobalWindow(new Record()); - WindowedValue two = WindowedValue.valueInGlobalWindow(new Record()); + WindowedValue one = WindowedValues.valueInGlobalWindow(new Record()); + WindowedValue two = WindowedValues.valueInGlobalWindow(new Record()); CommittedBundle root = factory.createRootBundle().add(one).add(two).commit(Instant.now()); @@ -76,9 +77,9 @@ public void bundleWorkingCoderSucceedsClonesOutput() { created .apply(WithKeys.of("foo")) .setCoder(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())); - WindowedValue> fooOne = WindowedValue.valueInGlobalWindow(KV.of("foo", 1)); + WindowedValue> fooOne = WindowedValues.valueInGlobalWindow(KV.of("foo", 1)); WindowedValue> fooThree = - WindowedValue.valueInGlobalWindow(KV.of("foo", 3)); + WindowedValues.valueInGlobalWindow(KV.of("foo", 3)); CommittedBundle> bundle = factory.createBundle(kvs).add(fooOne).add(fooThree).commit(Instant.now()); @@ -103,7 +104,7 @@ public void keyedBundleWorkingCoderSucceedsClonesOutput() { .setCoder(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())) .apply(GroupByKey.create()); WindowedValue>> foos = - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( KV.>of("foo", ImmutableList.of(1, 3))); CommittedBundle>> keyedBundle = factory @@ -127,7 +128,7 @@ public void bundleEncodeFailsAddFails() { thrown.expect(UserCodeException.class); thrown.expectCause(isA(CoderException.class)); thrown.expectMessage("Encode not allowed"); - bundle.add(WindowedValue.valueInGlobalWindow(new Record())); + bundle.add(WindowedValues.valueInGlobalWindow(new Record())); } @Test @@ -138,7 +139,7 @@ public void bundleDecodeFailsAddFails() { thrown.expect(UserCodeException.class); thrown.expectCause(isA(CoderException.class)); thrown.expectMessage("Decode not allowed"); - bundle.add(WindowedValue.valueInGlobalWindow(new Record())); + bundle.add(WindowedValues.valueInGlobalWindow(new Record())); } @Test @@ -150,7 +151,7 @@ public void keyedBundleEncodeFailsAddFails() { thrown.expect(UserCodeException.class); thrown.expectCause(isA(CoderException.class)); thrown.expectMessage("Encode not allowed"); - bundle.add(WindowedValue.valueInGlobalWindow(new Record())); + bundle.add(WindowedValues.valueInGlobalWindow(new Record())); } @Test @@ -162,7 +163,7 @@ public void keyedBundleDecodeFailsAddFails() { thrown.expect(UserCodeException.class); thrown.expectCause(isA(CoderException.class)); thrown.expectMessage("Decode not allowed"); - bundle.add(WindowedValue.valueInGlobalWindow(new Record())); + bundle.add(WindowedValues.valueInGlobalWindow(new Record())); } static class Record {} diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java index 87f6d773c9dd..a24d1c98c46f 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java @@ -31,11 +31,11 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.resourcehints.ResourceHints; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PValues; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.hamcrest.Matchers; @@ -85,7 +85,7 @@ public void getUncommittedElementsEqualInput() { CommittedBundle bundle = bundleFactory .createBundle(created) - .add(WindowedValue.valueInGlobalWindow(2)) + .add(WindowedValues.valueInGlobalWindow(2)) .commit(Instant.now()); CommittedResult> result = CommittedResult.create( diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTransformExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTransformExecutorTest.java index 196840c9d860..7884064bdd9b 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTransformExecutorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTransformExecutorTest.java @@ -40,9 +40,10 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.WithKeys; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.MoreExecutors; import org.hamcrest.Matchers; @@ -176,9 +177,9 @@ public TransformResult finishBundle() throws Exception { } }; - WindowedValue foo = WindowedValue.valueInGlobalWindow("foo"); - WindowedValue spam = WindowedValue.valueInGlobalWindow("spam"); - WindowedValue third = WindowedValue.valueInGlobalWindow("third"); + WindowedValue foo = WindowedValues.valueInGlobalWindow("foo"); + WindowedValue spam = WindowedValues.valueInGlobalWindow("spam"); + WindowedValue third = WindowedValues.valueInGlobalWindow("third"); CommittedBundle inputBundle = bundleFactory.createBundle(created).add(foo).add(spam).add(third).commit(Instant.now()); when(registry.forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator); @@ -222,7 +223,7 @@ public TransformResult finishBundle() throws Exception { } }; - WindowedValue foo = WindowedValue.valueInGlobalWindow("foo"); + WindowedValue foo = WindowedValues.valueInGlobalWindow("foo"); CommittedBundle inputBundle = bundleFactory.createBundle(created).add(foo).commit(Instant.now()); when(registry.forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator); @@ -295,8 +296,8 @@ public TransformResult finishBundle() throws Exception { } }; - WindowedValue fooElem = WindowedValue.valueInGlobalWindow("foo"); - WindowedValue barElem = WindowedValue.valueInGlobalWindow("bar"); + WindowedValue fooElem = WindowedValues.valueInGlobalWindow("foo"); + WindowedValue barElem = WindowedValues.valueInGlobalWindow("bar"); CommittedBundle inputBundle = bundleFactory.createBundle(created).add(fooElem).add(barElem).commit(Instant.now()); when(registry.forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator); @@ -334,7 +335,7 @@ public TransformResult finishBundle() throws Exception { } }; - WindowedValue fooBytes = WindowedValue.valueInGlobalWindow("foo"); + WindowedValue fooBytes = WindowedValues.valueInGlobalWindow("foo"); CommittedBundle inputBundle = bundleFactory.createBundle(created).add(fooBytes).commit(Instant.now()); when(registry.forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator); @@ -372,7 +373,7 @@ public TransformResult finishBundle() throws Exception { } }; - WindowedValue fooBytes = WindowedValue.valueInGlobalWindow("foo"); + WindowedValue fooBytes = WindowedValues.valueInGlobalWindow("foo"); CommittedBundle inputBundle = bundleFactory.createBundle(created).add(fooBytes).commit(Instant.now()); when(registry.forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java index 342b489ab12a..1a9cdc187bf9 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java @@ -33,7 +33,8 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Before; @@ -56,8 +57,8 @@ public void delegatesToUnderlying() throws Exception { ParDoEvaluator underlying = mock(ParDoEvaluator.class); TransformEvaluator evaluator = DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(underlying, lifecycleManager); - WindowedValue first = WindowedValue.valueInGlobalWindow(new Object()); - WindowedValue second = WindowedValue.valueInGlobalWindow(new Object()); + WindowedValue first = WindowedValues.valueInGlobalWindow(new Object()); + WindowedValue second = WindowedValues.valueInGlobalWindow(new Object()); evaluator.processElement(first); verify(underlying).processElement(first); @@ -82,7 +83,7 @@ public void removesOnExceptionInProcessElement() throws Exception { DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(underlying, lifecycleManager); try { - evaluator.processElement(WindowedValue.valueInGlobalWindow(new Object())); + evaluator.processElement(WindowedValues.valueInGlobalWindow(new Object())); } catch (Exception e) { assertThat(lifecycleManager.get(), not(Matchers.theInstance(original))); return; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java index b3f3ab529bdf..3e470c63c5dd 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java @@ -61,11 +61,12 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -153,13 +154,13 @@ public void writeToViewWriterThenReadReads() { for (Object materializedValue : materializeValuesFor(view.getPipeline().getOptions(), View.asIterable(), 1)) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, new Instant(1222), window, PaneInfo.ON_TIME_AND_ONLY_FIRING)); } for (Object materializedValue : materializeValuesFor(view.getPipeline().getOptions(), View.asIterable(), 2)) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, new Instant(8766L), second, @@ -175,7 +176,7 @@ public void writeToViewWriterThenReadReads() { for (Object materializedValue : materializeValuesFor(view.getPipeline().getOptions(), View.asIterable(), 4444)) { overwrittenValuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, new Instant(8677L), second, @@ -389,7 +390,7 @@ public void isDoneWithPartiallyDone() { // Impulse produces one element UncommittedBundle rootBundle = context.createBundle(impulse); - rootBundle.add(WindowedValue.valueInGlobalWindow(new byte[0])); + rootBundle.add(WindowedValues.valueInGlobalWindow(new byte[0])); CommittedResult handleResult = context.handleResult( null, diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java index 1aea14dac614..a8097d4898f1 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java @@ -30,9 +30,9 @@ import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.hamcrest.Matchers; import org.joda.time.Instant; @@ -74,15 +74,15 @@ public void testFlattenInMemoryEvaluator() throws Exception { TransformEvaluator rightSideEvaluator = factory.forApplication(flattenedProducer, rightBundle); - leftSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(1)); - rightSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(-1)); + leftSideEvaluator.processElement(WindowedValues.valueInGlobalWindow(1)); + rightSideEvaluator.processElement(WindowedValues.valueInGlobalWindow(-1)); leftSideEvaluator.processElement( - WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1024))); - leftSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(4, PaneInfo.NO_FIRING)); + WindowedValues.timestampedValueInGlobalWindow(2, new Instant(1024))); + leftSideEvaluator.processElement(WindowedValues.valueInGlobalWindow(4, PaneInfo.NO_FIRING)); rightSideEvaluator.processElement( - WindowedValue.valueInGlobalWindow(2, PaneInfo.ON_TIME_AND_ONLY_FIRING)); + WindowedValues.valueInGlobalWindow(2, PaneInfo.ON_TIME_AND_ONLY_FIRING)); rightSideEvaluator.processElement( - WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096))); + WindowedValues.timestampedValueInGlobalWindow(-4, new Instant(-4096))); TransformResult rightSideResult = rightSideEvaluator.finishBundle(); TransformResult leftSideResult = leftSideEvaluator.finishBundle(); @@ -99,15 +99,15 @@ public void testFlattenInMemoryEvaluator() throws Exception { assertThat( flattenedLeftBundle.commit(Instant.now()).getElements(), containsInAnyOrder( - WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1024)), - WindowedValue.valueInGlobalWindow(4, PaneInfo.NO_FIRING), - WindowedValue.valueInGlobalWindow(1))); + WindowedValues.timestampedValueInGlobalWindow(2, new Instant(1024)), + WindowedValues.valueInGlobalWindow(4, PaneInfo.NO_FIRING), + WindowedValues.valueInGlobalWindow(1))); assertThat( flattenedRightBundle.commit(Instant.now()).getElements(), containsInAnyOrder( - WindowedValue.valueInGlobalWindow(2, PaneInfo.ON_TIME_AND_ONLY_FIRING), - WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096)), - WindowedValue.valueInGlobalWindow(-1))); + WindowedValues.valueInGlobalWindow(2, PaneInfo.ON_TIME_AND_ONLY_FIRING), + WindowedValues.timestampedValueInGlobalWindow(-4, new Instant(-4096)), + WindowedValues.valueInGlobalWindow(-1))); } @Test diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java index 63b6fb5c29e9..e6057f3b4d87 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java @@ -31,9 +31,10 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashMultiset; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Multiset; @@ -91,12 +92,12 @@ public void testInMemoryEvaluator() throws Exception { new GroupByKeyOnlyEvaluatorFactory(evaluationContext) .forApplication(DirectGraphs.getProducer(groupedKvs), inputBundle); - evaluator.processElement(WindowedValue.valueInGlobalWindow(firstFoo)); - evaluator.processElement(WindowedValue.valueInGlobalWindow(secondFoo)); - evaluator.processElement(WindowedValue.valueInGlobalWindow(thirdFoo)); - evaluator.processElement(WindowedValue.valueInGlobalWindow(firstBar)); - evaluator.processElement(WindowedValue.valueInGlobalWindow(secondBar)); - evaluator.processElement(WindowedValue.valueInGlobalWindow(firstBaz)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(firstFoo)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(secondFoo)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(thirdFoo)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(firstBar)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(secondBar)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(firstBaz)); evaluator.finishBundle(); @@ -107,9 +108,9 @@ public void testInMemoryEvaluator() throws Exception { KeyedWorkItems.elementsWorkItem( "foo", ImmutableSet.of( - WindowedValue.valueInGlobalWindow(-1), - WindowedValue.valueInGlobalWindow(1), - WindowedValue.valueInGlobalWindow(3))), + WindowedValues.valueInGlobalWindow(-1), + WindowedValues.valueInGlobalWindow(1), + WindowedValues.valueInGlobalWindow(3))), keyCoder))); assertThat( barBundle.commit(Instant.now()).getElements(), @@ -118,15 +119,15 @@ public void testInMemoryEvaluator() throws Exception { KeyedWorkItems.elementsWorkItem( "bar", ImmutableSet.of( - WindowedValue.valueInGlobalWindow(12), - WindowedValue.valueInGlobalWindow(22))), + WindowedValues.valueInGlobalWindow(12), + WindowedValues.valueInGlobalWindow(22))), keyCoder))); assertThat( bazBundle.commit(Instant.now()).getElements(), contains( new KeyedWorkItemMatcher<>( KeyedWorkItems.elementsWorkItem( - "baz", ImmutableSet.of(WindowedValue.valueInGlobalWindow(Integer.MAX_VALUE))), + "baz", ImmutableSet.of(WindowedValues.valueInGlobalWindow(Integer.MAX_VALUE))), keyCoder))); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java index 135f42b01323..9fd07742627d 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java @@ -31,9 +31,10 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashMultiset; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Multiset; @@ -90,12 +91,12 @@ public void testInMemoryEvaluator() throws Exception { new GroupByKeyOnlyEvaluatorFactory(evaluationContext) .forApplication(DirectGraphs.getProducer(groupedKvs), inputBundle); - evaluator.processElement(WindowedValue.valueInGlobalWindow(firstFoo)); - evaluator.processElement(WindowedValue.valueInGlobalWindow(secondFoo)); - evaluator.processElement(WindowedValue.valueInGlobalWindow(thirdFoo)); - evaluator.processElement(WindowedValue.valueInGlobalWindow(firstBar)); - evaluator.processElement(WindowedValue.valueInGlobalWindow(secondBar)); - evaluator.processElement(WindowedValue.valueInGlobalWindow(firstBaz)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(firstFoo)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(secondFoo)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(thirdFoo)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(firstBar)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(secondBar)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(firstBaz)); evaluator.finishBundle(); @@ -106,9 +107,9 @@ public void testInMemoryEvaluator() throws Exception { KeyedWorkItems.elementsWorkItem( "foo", ImmutableSet.of( - WindowedValue.valueInGlobalWindow(-1), - WindowedValue.valueInGlobalWindow(1), - WindowedValue.valueInGlobalWindow(3))), + WindowedValues.valueInGlobalWindow(-1), + WindowedValues.valueInGlobalWindow(1), + WindowedValues.valueInGlobalWindow(3))), keyCoder))); assertThat( barBundle.commit(Instant.now()).getElements(), @@ -117,15 +118,15 @@ public void testInMemoryEvaluator() throws Exception { KeyedWorkItems.elementsWorkItem( "bar", ImmutableSet.of( - WindowedValue.valueInGlobalWindow(12), - WindowedValue.valueInGlobalWindow(22))), + WindowedValues.valueInGlobalWindow(12), + WindowedValues.valueInGlobalWindow(22))), keyCoder))); assertThat( bazBundle.commit(Instant.now()).getElements(), contains( new KeyedWorkItemMatcher<>( KeyedWorkItems.elementsWorkItem( - "baz", ImmutableSet.of(WindowedValue.valueInGlobalWindow(Integer.MAX_VALUE))), + "baz", ImmutableSet.of(WindowedValues.valueInGlobalWindow(Integer.MAX_VALUE))), keyCoder))); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java index 5dcfa90cb4c7..bdfbb04d78c8 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java @@ -30,8 +30,9 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.IllegalMutationException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.joda.time.Instant; import org.junit.Before; import org.junit.Rule; @@ -65,11 +66,11 @@ public void setup() { public void rootBundleSucceeds() { UncommittedBundle root = factory.createRootBundle(); byte[] array = new byte[] {0, 1, 2}; - root.add(WindowedValue.valueInGlobalWindow(array)); + root.add(WindowedValues.valueInGlobalWindow(array)); CommittedBundle committed = root.commit(Instant.now()); assertThat( - committed.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(array))); + committed.getElements(), containsInAnyOrder(WindowedValues.valueInGlobalWindow(array))); } @Test @@ -78,7 +79,7 @@ public void noMutationKeyedBundleSucceeds() { factory.createKeyedBundle(StructuralKey.of("mykey", StringUtf8Coder.of()), transformed); WindowedValue windowedArray = - WindowedValue.of( + WindowedValues.of( new byte[] {4, 8, 12}, new Instant(891L), new IntervalWindow(new Instant(0), new Instant(1000)), @@ -94,7 +95,7 @@ public void noMutationCreateBundleSucceeds() { UncommittedBundle intermediate = factory.createBundle(transformed); WindowedValue windowedArray = - WindowedValue.of( + WindowedValues.of( new byte[] {4, 8, 12}, new Instant(891L), new IntervalWindow(new Instant(0), new Instant(1000)), @@ -113,7 +114,7 @@ public void mutationBeforeAddKeyedBundleSucceeds() { byte[] array = new byte[] {4, 8, 12}; array[0] = Byte.MAX_VALUE; WindowedValue windowedArray = - WindowedValue.of( + WindowedValues.of( array, new Instant(891L), new IntervalWindow(new Instant(0), new Instant(1000)), @@ -130,7 +131,7 @@ public void mutationBeforeAddCreateBundleSucceeds() { byte[] array = new byte[] {4, 8, 12}; WindowedValue windowedArray = - WindowedValue.of( + WindowedValues.of( array, new Instant(891L), new IntervalWindow(new Instant(0), new Instant(1000)), @@ -149,7 +150,7 @@ public void mutationAfterAddKeyedBundleThrows() { byte[] array = new byte[] {4, 8, 12}; WindowedValue windowedArray = - WindowedValue.of( + WindowedValues.of( array, new Instant(891L), new IntervalWindow(new Instant(0), new Instant(1000)), @@ -168,7 +169,7 @@ public void mutationAfterAddCreateBundleThrows() { byte[] array = new byte[] {4, 8, 12}; WindowedValue windowedArray = - WindowedValue.of( + WindowedValues.of( array, new Instant(891L), new IntervalWindow(new Instant(0), new Instant(1000)), diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java index 3a574102f664..60db1eb85e02 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java @@ -28,8 +28,9 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.util.IllegalMutationException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.joda.time.Instant; import org.junit.Before; import org.junit.Rule; @@ -71,7 +72,7 @@ public void processElement(ProcessContext c) throws Exception { @Test public void unchangedSucceeds() { - WindowedValue element = WindowedValue.valueInGlobalWindow("bar".getBytes(UTF_8)); + WindowedValue element = WindowedValues.valueInGlobalWindow("bar".getBytes(UTF_8)); CommittedBundle elements = bundleFactory.createBundle(pcollection).add(element).commit(Instant.now()); @@ -86,7 +87,7 @@ public void unchangedSucceeds() { @Test public void mutatedDuringProcessElementThrows() { - WindowedValue element = WindowedValue.valueInGlobalWindow("bar".getBytes(UTF_8)); + WindowedValue element = WindowedValues.valueInGlobalWindow("bar".getBytes(UTF_8)); CommittedBundle elements = bundleFactory.createBundle(pcollection).add(element).commit(Instant.now()); @@ -107,7 +108,7 @@ public void mutatedDuringProcessElementThrows() { @Test public void mutatedAfterProcessElementFails() { - WindowedValue element = WindowedValue.valueInGlobalWindow("bar".getBytes(UTF_8)); + WindowedValue element = WindowedValues.valueInGlobalWindow("bar".getBytes(UTF_8)); CommittedBundle elements = bundleFactory.createBundle(pcollection).add(element).commit(Instant.now()); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java index 5b008c3235ef..6b84908d626e 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java @@ -37,9 +37,10 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.hamcrest.Matcher; import org.joda.time.Duration; @@ -131,9 +132,9 @@ public void getElementsBeforeAddShouldReturnEmptyIterable() { @Test public void getElementsAfterAddShouldReturnAddedElements() { - WindowedValue firstValue = WindowedValue.valueInGlobalWindow(1); + WindowedValue firstValue = WindowedValues.valueInGlobalWindow(1); WindowedValue secondValue = - WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1000L)); + WindowedValues.timestampedValueInGlobalWindow(2, new Instant(1000L)); afterCommitGetElementsShouldHaveAddedElements(ImmutableList.of(firstValue, secondValue)); } @@ -141,7 +142,7 @@ public void getElementsAfterAddShouldReturnAddedElements() { @Test public void addElementsAtEndOfTimeThrows() { Instant timestamp = BoundedWindow.TIMESTAMP_MAX_VALUE; - WindowedValue value = WindowedValue.timestampedValueInGlobalWindow(1, timestamp); + WindowedValue value = WindowedValues.timestampedValueInGlobalWindow(1, timestamp); UncommittedBundle bundle = bundleFactory.createRootBundle(); thrown.expect(IllegalArgumentException.class); @@ -152,7 +153,7 @@ public void addElementsAtEndOfTimeThrows() { @Test public void addElementsPastEndOfTimeThrows() { Instant timestamp = BoundedWindow.TIMESTAMP_MAX_VALUE.plus(Duration.standardMinutes(2)); - WindowedValue value = WindowedValue.timestampedValueInGlobalWindow(1, timestamp); + WindowedValue value = WindowedValues.timestampedValueInGlobalWindow(1, timestamp); UncommittedBundle bundle = bundleFactory.createRootBundle(); thrown.expect(IllegalArgumentException.class); @@ -163,21 +164,21 @@ public void addElementsPastEndOfTimeThrows() { @SuppressWarnings("unchecked") @Test public void withElementsShouldReturnIndependentBundle() { - WindowedValue firstValue = WindowedValue.valueInGlobalWindow(1); + WindowedValue firstValue = WindowedValues.valueInGlobalWindow(1); WindowedValue secondValue = - WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1000L)); + WindowedValues.timestampedValueInGlobalWindow(2, new Instant(1000L)); CommittedBundle committed = afterCommitGetElementsShouldHaveAddedElements(ImmutableList.of(firstValue, secondValue)); WindowedValue firstReplacement = - WindowedValue.of( + WindowedValues.of( 9, new Instant(2048L), new IntervalWindow(new Instant(2044L), Instant.now()), PaneInfo.NO_FIRING); WindowedValue secondReplacement = - WindowedValue.timestampedValueInGlobalWindow(-1, Instant.now()); + WindowedValues.timestampedValueInGlobalWindow(-1, Instant.now()); CommittedBundle withed = committed.withElements(ImmutableList.of(firstReplacement, secondReplacement)); @@ -194,23 +195,25 @@ public void withElementsShouldReturnIndependentBundle() { @Test public void addAfterCommitShouldThrowException() { UncommittedBundle bundle = bundleFactory.createRootBundle(); - bundle.add(WindowedValue.valueInGlobalWindow(1)); + bundle.add(WindowedValues.valueInGlobalWindow(1)); CommittedBundle firstCommit = bundle.commit(Instant.now()); - assertThat(firstCommit.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(1))); + assertThat( + firstCommit.getElements(), containsInAnyOrder(WindowedValues.valueInGlobalWindow(1))); thrown.expect(IllegalStateException.class); thrown.expectMessage("3"); thrown.expectMessage("committed"); - bundle.add(WindowedValue.valueInGlobalWindow(3)); + bundle.add(WindowedValues.valueInGlobalWindow(3)); } @Test public void commitAfterCommitShouldThrowException() { UncommittedBundle bundle = bundleFactory.createRootBundle(); - bundle.add(WindowedValue.valueInGlobalWindow(1)); + bundle.add(WindowedValues.valueInGlobalWindow(1)); CommittedBundle firstCommit = bundle.commit(Instant.now()); - assertThat(firstCommit.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(1))); + assertThat( + firstCommit.getElements(), containsInAnyOrder(WindowedValues.valueInGlobalWindow(1))); thrown.expect(IllegalStateException.class); thrown.expectMessage("committed"); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImpulseEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImpulseEvaluatorFactoryTest.java index 758b6811c496..5c3718b14456 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImpulseEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImpulseEvaluatorFactoryTest.java @@ -32,8 +32,9 @@ import org.apache.beam.sdk.transforms.Impulse; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.joda.time.Instant; import org.junit.Before; @@ -64,7 +65,7 @@ public void testImpulse() throws Exception { ImpulseEvaluatorFactory factory = new ImpulseEvaluatorFactory(context); - WindowedValue inputShard = WindowedValue.valueInGlobalWindow(new ImpulseShard()); + WindowedValue inputShard = WindowedValues.valueInGlobalWindow(new ImpulseShard()); CommittedBundle inputShardBundle = bundleFactory.createRootBundle().add(inputShard).commit(Instant.now()); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java index a228da6eff54..95338e4bf330 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java @@ -40,11 +40,12 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; import org.junit.Before; @@ -117,7 +118,7 @@ public void unkeyedInputWithKeyPreserving() { KvCoder.of( StringUtf8Coder.of(), IterableCoder.of( - WindowedValue.getValueOnlyCoder( + WindowedValues.getValueOnlyCoder( KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())))))); PCollection>> unkeyed = @@ -141,7 +142,7 @@ public void keyedInputWithKeyPreserving() { Create.of( KV.of( "hello", - WindowedValue.of( + WindowedValues.of( KV.of("hello", 3), new Instant(0), new IntervalWindow(new Instant(0), new Instant(9)), @@ -149,7 +150,7 @@ public void keyedInputWithKeyPreserving() { .withCoder( KvCoder.of( StringUtf8Coder.of(), - WindowedValue.getValueOnlyCoder( + WindowedValues.getValueOnlyCoder( KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))))); TupleTag>> keyedTag = new TupleTag<>(); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java index d7f82c4e045b..4429385d7975 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java @@ -45,11 +45,12 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.IdentitySideInputWindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -102,11 +103,11 @@ public void sideInputsNotReadyResultHasUnprocessedElements() { ParDoEvaluator evaluator = createEvaluator(singletonView, fn, inputPc, output); IntervalWindow nonGlobalWindow = new IntervalWindow(new Instant(0), new Instant(10_000L)); - WindowedValue first = WindowedValue.valueInGlobalWindow(3); + WindowedValue first = WindowedValues.valueInGlobalWindow(3); WindowedValue second = - WindowedValue.of(2, new Instant(1234L), nonGlobalWindow, PaneInfo.NO_FIRING); + WindowedValues.of(2, new Instant(1234L), nonGlobalWindow, PaneInfo.NO_FIRING); WindowedValue third = - WindowedValue.of( + WindowedValues.of( 1, new Instant(2468L), ImmutableList.of(nonGlobalWindow, GlobalWindow.INSTANCE), @@ -120,14 +121,14 @@ public void sideInputsNotReadyResultHasUnprocessedElements() { assertThat( result.getUnprocessedElements(), Matchers.>containsInAnyOrder( - second, WindowedValue.of(1, new Instant(2468L), nonGlobalWindow, PaneInfo.NO_FIRING))); + second, WindowedValues.of(1, new Instant(2468L), nonGlobalWindow, PaneInfo.NO_FIRING))); assertThat(result.getOutputBundles(), Matchers.contains(outputBundle)); assertThat(fn.processed, containsInAnyOrder(1, 3)); assertThat( Iterables.getOnlyElement(result.getOutputBundles()).commit(Instant.now()).getElements(), containsInAnyOrder( first.withValue(8), - WindowedValue.timestampedValueInGlobalWindow(6, new Instant(2468L)))); + WindowedValues.timestampedValueInGlobalWindow(6, new Instant(2468L)))); } private ParDoEvaluator createEvaluator( diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java index cb8d655aa602..2f5b48192716 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java @@ -43,11 +43,12 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.MoreExecutors; import org.joda.time.Duration; @@ -126,13 +127,13 @@ public void getAfterWriteReturnsPaneInWindow() throws Exception { for (Object materializedValue : materializeValuesFor(mapView.getPipeline().getOptions(), View.asMap(), KV.of("one", 1))) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, new Instant(1L), FIRST_WINDOW, PaneInfo.ON_TIME_AND_ONLY_FIRING)); } for (Object materializedValue : materializeValuesFor(mapView.getPipeline().getOptions(), View.asMap(), KV.of("two", 2))) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, new Instant(20L), FIRST_WINDOW, PaneInfo.ON_TIME_AND_ONLY_FIRING)); } container.write(mapView, valuesBuilder.build()); @@ -150,7 +151,7 @@ public void getReturnsLatestPaneInWindow() throws Exception { for (Object materializedValue : materializeValuesFor(mapView.getPipeline().getOptions(), View.asMap(), KV.of("one", 1))) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, new Instant(1L), SECOND_WINDOW, @@ -159,7 +160,7 @@ public void getReturnsLatestPaneInWindow() throws Exception { for (Object materializedValue : materializeValuesFor(mapView.getPipeline().getOptions(), View.asMap(), KV.of("two", 2))) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, new Instant(20L), SECOND_WINDOW, @@ -177,7 +178,7 @@ public void getReturnsLatestPaneInWindow() throws Exception { for (Object materializedValue : materializeValuesFor(mapView.getPipeline().getOptions(), View.asMap(), KV.of("three", 3))) { overwriteValuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, new Instant(300L), SECOND_WINDOW, @@ -232,7 +233,7 @@ public void writeForMultipleElementsInDifferentWindowsSucceeds() throws Exceptio for (Object materializedValue : materializeValuesFor(singletonView.getPipeline().getOptions(), View.asSingleton(), 2.875)) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, FIRST_WINDOW.maxTimestamp().minus(Duration.millis(200L)), FIRST_WINDOW, @@ -241,7 +242,7 @@ public void writeForMultipleElementsInDifferentWindowsSucceeds() throws Exceptio for (Object materializedValue : materializeValuesFor(singletonView.getPipeline().getOptions(), View.asSingleton(), 4.125)) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, SECOND_WINDOW.maxTimestamp().minus(Duration.millis(2_000_000L)), SECOND_WINDOW, @@ -266,7 +267,7 @@ public void writeForMultipleIdenticalElementsInSameWindowSucceeds() throws Excep for (Object materializedValue : materializeValuesFor(iterableView.getPipeline().getOptions(), View.asIterable(), 44, 44)) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, FIRST_WINDOW.maxTimestamp().minus(Duration.millis(200L)), FIRST_WINDOW, @@ -287,7 +288,7 @@ public void writeForElementInMultipleWindowsSucceeds() throws Exception { for (Object materializedValue : materializeValuesFor(singletonView.getPipeline().getOptions(), View.asSingleton(), 2.875)) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, FIRST_WINDOW.maxTimestamp().minus(Duration.millis(200L)), ImmutableList.of(FIRST_WINDOW, SECOND_WINDOW), @@ -312,7 +313,7 @@ public void finishDoesNotOverwriteWrittenElements() throws Exception { for (Object materializedValue : materializeValuesFor(mapView.getPipeline().getOptions(), View.asMap(), KV.of("one", 1))) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, new Instant(1L), SECOND_WINDOW, @@ -321,7 +322,7 @@ public void finishDoesNotOverwriteWrittenElements() throws Exception { for (Object materializedValue : materializeValuesFor(mapView.getPipeline().getOptions(), View.asMap(), KV.of("two", 2))) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, new Instant(20L), SECOND_WINDOW, @@ -372,7 +373,7 @@ public void isReadyForSomeNotReadyViewsFalseUntilElements() { for (Object materializedValue : materializeValuesFor(mapView.getPipeline().getOptions(), View.asMap(), KV.of("one", 1))) { mapValuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, SECOND_WINDOW.maxTimestamp().minus(Duration.millis(100L)), SECOND_WINDOW, @@ -391,7 +392,7 @@ public void isReadyForSomeNotReadyViewsFalseUntilElements() { for (Object materializedValue : materializeValuesFor(mapView.getPipeline().getOptions(), View.asMap(), KV.of("too", 2))) { newMapValuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, FIRST_WINDOW.maxTimestamp().minus(Duration.millis(100L)), FIRST_WINDOW, @@ -405,7 +406,7 @@ public void isReadyForSomeNotReadyViewsFalseUntilElements() { for (Object materializedValue : materializeValuesFor(singletonView.getPipeline().getOptions(), View.asSingleton(), 1.25)) { singletonValuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, SECOND_WINDOW.maxTimestamp().minus(Duration.millis(100L)), SECOND_WINDOW, diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java index 7746e48317d5..cd3f0ef925e4 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java @@ -60,7 +60,6 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; @@ -69,6 +68,8 @@ import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -203,7 +204,7 @@ public void process(ProcessContext c) {} // depend on the window. String key = "hello"; WindowedValue> firstKv = - WindowedValue.of(KV.of(key, 1), new Instant(3), firstWindow, PaneInfo.NO_FIRING); + WindowedValues.of(KV.of(key, 1), new Instant(3), firstWindow, PaneInfo.NO_FIRING); WindowedValue>> gbkOutputElement = firstKv.withValue( diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java index 188c5e66b1c4..164aad503ff9 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java @@ -32,9 +32,10 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.hamcrest.Matchers; import org.joda.time.Duration; @@ -159,18 +160,18 @@ public void producesElementsInSequence() throws Exception { .commit(Instant.now()) .getElements(), Matchers.containsInAnyOrder( - WindowedValue.valueInGlobalWindow(1), - WindowedValue.valueInGlobalWindow(2), - WindowedValue.valueInGlobalWindow(3))); + WindowedValues.valueInGlobalWindow(1), + WindowedValues.valueInGlobalWindow(2), + WindowedValues.valueInGlobalWindow(3))); assertThat( Iterables.getOnlyElement(thirdResult.getOutputBundles()) .commit(Instant.now()) .getElements(), Matchers.containsInAnyOrder( - WindowedValue.valueInGlobalWindow(4), - WindowedValue.valueInGlobalWindow(5), - WindowedValue.valueInGlobalWindow(6))); + WindowedValues.valueInGlobalWindow(4), + WindowedValues.valueInGlobalWindow(5), + WindowedValues.valueInGlobalWindow(6))); assertThat(fifthResult.getOutputBundles(), Matchers.emptyIterable()); assertThat(fifthResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE)); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java index d31ce1f78ba5..5413a694e92b 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java @@ -72,11 +72,12 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.VarInt; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ContiguousSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.DiscreteDomain; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -311,7 +312,7 @@ public void evaluatorReusesReaderAndClosesAtTheEnd() throws Exception { when(context.createBundle(pcollection)).thenReturn(output); WindowedValue> shard = - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( UnboundedSourceShard.unstarted(source, NeverDeduplicator.create())); CommittedBundle> inputBundle = bundleFactory @@ -356,7 +357,7 @@ public void evaluatorClosesReaderAndResumesFromCheckpoint() throws Exception { when(context.createBundle(pcollection)).thenReturn(output); WindowedValue> shard = - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( UnboundedSourceShard.unstarted(source, NeverDeduplicator.create())); CommittedBundle> inputBundle = bundleFactory @@ -402,7 +403,7 @@ public void evaluatorThrowsInCloseRethrows() throws Exception { when(context.createBundle(pcollection)).thenReturn(output); WindowedValue> shard = - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( UnboundedSourceShard.unstarted(source, NeverDeduplicator.create())); CommittedBundle> inputBundle = bundleFactory @@ -469,7 +470,7 @@ private void processElement(final TestUnboundedSource source) throws Exc final UnboundedSourceShard shard = UnboundedSourceShard.of(source, new NeverDeduplicator(), reader, null); final WindowedValue> value = - WindowedValue.of( + WindowedValues.of( shard, BoundedWindow.TIMESTAMP_MAX_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); TestUnboundedSource.readerClosedCount = 0; evaluator.processElement(value); @@ -480,7 +481,7 @@ private void processElement(final TestUnboundedSource source) throws Exc * is the epoch offset by the value of the element. */ private static WindowedValue tgw(Long elem) { - return WindowedValue.timestampedValueInGlobalWindow(elem, new Instant(elem)); + return WindowedValues.timestampedValueInGlobalWindow(elem, new Instant(elem)); } private static class LongToInstantFn implements SerializableFunction { diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java index 91e094927e70..07f0136fff39 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java @@ -33,9 +33,10 @@ import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.WithKeys; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Instant; import org.junit.Rule; @@ -72,14 +73,14 @@ public void testInMemoryEvaluator() throws Exception { TransformEvaluator> evaluator = new ViewEvaluatorFactory(context).forApplication(producer, inputBundle); - evaluator.processElement(WindowedValue.valueInGlobalWindow(ImmutableList.of("foo", "bar"))); + evaluator.processElement(WindowedValues.valueInGlobalWindow(ImmutableList.of("foo", "bar"))); assertThat(viewWriter.latest, nullValue()); evaluator.finishBundle(); assertThat( viewWriter.latest, containsInAnyOrder( - WindowedValue.valueInGlobalWindow("foo"), WindowedValue.valueInGlobalWindow("bar"))); + WindowedValues.valueInGlobalWindow("foo"), WindowedValues.valueInGlobalWindow("bar"))); } private static class TestViewWriter implements PCollectionViewWriter { diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java index 847acd493553..785c59bae84f 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java @@ -61,11 +61,12 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -320,12 +321,12 @@ public void getWatermarkMultiIdenticalInput() { CommittedBundle root = bundleFactory .createRootBundle() - .add(WindowedValue.valueInGlobalWindow(null)) + .add(WindowedValues.valueInGlobalWindow(null)) .commit(clock.now()); CommittedBundle createBundle = bundleFactory .createBundle(impulse) - .add(WindowedValue.timestampedValueInGlobalWindow(new byte[1], new Instant(33536))) + .add(WindowedValues.timestampedValueInGlobalWindow(new byte[1], new Instant(33536))) .commit(clock.now()); Map, Collection>> initialInputs = @@ -485,14 +486,15 @@ public void updateWatermarkWithKeyedWatermarkHolds() { CommittedBundle firstKeyBundle = bundleFactory .createKeyedBundle(StructuralKey.of("Odd", StringUtf8Coder.of()), impulse) - .add(WindowedValue.timestampedValueInGlobalWindow(new byte[1], new Instant(1_000_000L))) - .add(WindowedValue.timestampedValueInGlobalWindow(new byte[3], new Instant(-1000L))) + .add( + WindowedValues.timestampedValueInGlobalWindow(new byte[1], new Instant(1_000_000L))) + .add(WindowedValues.timestampedValueInGlobalWindow(new byte[3], new Instant(-1000L))) .commit(clock.now()); CommittedBundle secondKeyBundle = bundleFactory .createKeyedBundle(StructuralKey.of("Even", StringUtf8Coder.of()), impulse) - .add(WindowedValue.timestampedValueInGlobalWindow(new byte[2], new Instant(1234L))) + .add(WindowedValues.timestampedValueInGlobalWindow(new byte[2], new Instant(1234L))) .commit(clock.now()); manager.updateWatermarks( @@ -648,11 +650,11 @@ public void updateWatermarkWithHoldsShouldBeMonotonic() { @Test public void updateWatermarkWithUnprocessedElements() { - WindowedValue first = WindowedValue.valueInGlobalWindow(new byte[1]); + WindowedValue first = WindowedValues.valueInGlobalWindow(new byte[1]); WindowedValue second = - WindowedValue.timestampedValueInGlobalWindow(new byte[2], new Instant(-1000L)); + WindowedValues.timestampedValueInGlobalWindow(new byte[2], new Instant(-1000L)); WindowedValue third = - WindowedValue.timestampedValueInGlobalWindow(new byte[3], new Instant(1234L)); + WindowedValues.timestampedValueInGlobalWindow(new byte[3], new Instant(1234L)); CommittedBundle impulseBundle = bundleFactory.createBundle(impulse).add(first).add(second).add(third).commit(clock.now()); @@ -683,12 +685,12 @@ public void updateWatermarkWithUnprocessedElements() { @Test public void updateWatermarkWithCompletedElementsNotPending() { WindowedValue first = - WindowedValue.timestampedValueInGlobalWindow(new byte[1], new Instant(22)); + WindowedValues.timestampedValueInGlobalWindow(new byte[1], new Instant(22)); CommittedBundle impulseBundle = bundleFactory.createBundle(impulse).add(first).commit(clock.now()); WindowedValue second = - WindowedValue.timestampedValueInGlobalWindow(new byte[2], new Instant(22)); + WindowedValues.timestampedValueInGlobalWindow(new byte[2], new Instant(22)); CommittedBundle neverImpulseBundle = bundleFactory.createBundle(impulse).add(second).commit(clock.now()); @@ -795,14 +797,14 @@ public void updateWatermarkWithDifferentWindowedValueInstances() { Collections.>singleton( bundleFactory .createBundle(impulse) - .add(WindowedValue.valueInGlobalWindow(new byte[1])) + .add(WindowedValues.valueInGlobalWindow(new byte[1])) .commit(Instant.now())), BoundedWindow.TIMESTAMP_MAX_VALUE); CommittedBundle impulseBundle = bundleFactory .createBundle(impulse) - .add(WindowedValue.valueInGlobalWindow(new byte[1])) + .add(WindowedValues.valueInGlobalWindow(new byte[1])) .commit(Instant.now()); manager.updateWatermarks( impulseBundle, @@ -1792,7 +1794,7 @@ private final CommittedBundle timestampedBundle( UncommittedBundle bundle = bundleFactory.createBundle(pc); for (TimestampedValue value : values) { bundle.add( - WindowedValue.timestampedValueInGlobalWindow(value.getValue(), value.getTimestamp())); + WindowedValues.timestampedValueInGlobalWindow(value.getValue(), value.getTimestamp())); } return bundle.commit(BoundedWindow.TIMESTAMP_MAX_VALUE); } @@ -1806,7 +1808,7 @@ private final CommittedBundle multiWindowedBundle(PCollection pc, T... new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(0))); for (T value : values) { bundle.add( - WindowedValue.of(value, BoundedWindow.TIMESTAMP_MIN_VALUE, windows, PaneInfo.NO_FIRING)); + WindowedValues.of(value, BoundedWindow.TIMESTAMP_MIN_VALUE, windows, PaneInfo.NO_FIRING)); } return bundle.commit(BoundedWindow.TIMESTAMP_MAX_VALUE); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java index b334c8431683..ccf6b53fc8fb 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java @@ -41,8 +41,9 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -70,11 +71,11 @@ public class WindowEvaluatorFactoryTest { private BundleFactory bundleFactory; private WindowedValue valueInGlobalWindow = - WindowedValue.timestampedValueInGlobalWindow(3L, new Instant(2L)); + WindowedValues.timestampedValueInGlobalWindow(3L, new Instant(2L)); private final PaneInfo intervalWindowPane = PaneInfo.createPane(false, false, Timing.LATE, 3, 2); private WindowedValue valueInIntervalWindow = - WindowedValue.of( + WindowedValues.of( 2L, new Instant(-10L), new IntervalWindow(new Instant(-100), EPOCH), intervalWindowPane); private IntervalWindow intervalWindow1 = @@ -86,7 +87,7 @@ public class WindowEvaluatorFactoryTest { private final PaneInfo multiWindowPane = PaneInfo.createPane(false, true, Timing.ON_TIME, 3, 0); private WindowedValue valueInGlobalAndTwoIntervalWindows = - WindowedValue.of( + WindowedValues.of( 1L, EPOCH.plus(Duration.standardDays(3)), ImmutableList.of(GlobalWindow.INSTANCE, intervalWindow1, intervalWindow2), diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java index 090923a1cb92..dd0ced6ba11c 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java @@ -58,8 +58,6 @@ import org.apache.beam.sdk.transforms.join.UnionCoder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.construction.NativeTransforms; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.RehydratedComponents; @@ -70,6 +68,9 @@ import org.apache.beam.sdk.util.construction.graph.PipelineNode.PTransformNode; import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; @@ -361,7 +362,7 @@ private static void translateExecutableStage( if (stagePayload.getUserStatesCount() > 0 || stagePayload.getTimersCount() > 0) { Coder valueCoder = - ((WindowedValue.FullWindowedValueCoder) windowedInputCoder).getValueCoder(); + ((WindowedValues.FullWindowedValueCoder) windowedInputCoder).getValueCoder(); // Stateful stages are only allowed of KV input to be able to group on the key if (!(valueCoder instanceof KvCoder)) { throw new IllegalStateException( @@ -437,7 +438,7 @@ private static void translateFlatten( }) .returns( new CoderTypeInformation<>( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( (Coder) VoidCoder.of(), GlobalWindow.Coder.INSTANCE), context.getPipelineOptions())); } else { @@ -508,7 +509,7 @@ private static void translateGroupByKey( CoderRegistry.createDefault(), inputElementCoder.getValueCoder()); Coder>>> outputCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(inputElementCoder.getKeyCoder(), accumulatorCoder), windowingStrategy.getWindowFn().windowCoder()); @@ -554,7 +555,7 @@ private static void translateImpulse( PTransformNode transform, RunnerApi.Pipeline pipeline, BatchTranslationContext context) { TypeInformation> typeInformation = new CoderTypeInformation<>( - WindowedValue.getFullCoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE), context.getPipelineOptions()); DataSource> dataSource = new DataSource<>( diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java index 383730daeef8..a56dc9feee8f 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java @@ -63,7 +63,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CreatePCollectionViewTranslation; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.ParDoTranslation; @@ -76,6 +75,8 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; @@ -294,7 +295,7 @@ public void translateNode( inputCoder.getKeyCoder(), windowingStrategy.getWindowFn().windowCoder())); final TypeInformation>>> outputTypeInfo = new CoderTypeInformation<>( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( inputCoder.getKeyCoder(), IterableCoder.of(inputCoder.getValueCoder())), windowingStrategy.getWindowFn().windowCoder()), @@ -345,7 +346,7 @@ public void translateNode( TypeInformation>>> partialReduceTypeInfo = new CoderTypeInformation<>( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder), windowingStrategy.getWindowFn().windowCoder()), context.getPipelineOptions()); @@ -411,8 +412,8 @@ public void translateNode( outputType, FlinkIdentityFunction.of(), getCurrentTransformName(context)); - WindowedValue.WindowedValueCoder> kvWvCoder = - (WindowedValue.WindowedValueCoder>) outputType.getCoder(); + WindowedValues.WindowedValueCoder> kvWvCoder = + (WindowedValues.WindowedValueCoder>) outputType.getCoder(); KvCoder kvCoder = (KvCoder) kvWvCoder.getValueCoder(); DataSet>> reshuffle = retypedDataSet @@ -648,7 +649,8 @@ public void translateNode( TypeInformation> typeInformation = new CoderTypeInformation<>( - WindowedValue.getFullCoder(unionCoder, windowingStrategy.getWindowFn().windowCoder()), + WindowedValues.getFullCoder( + unionCoder, windowingStrategy.getWindowFn().windowCoder()), context.getPipelineOptions()); List> sideInputs; @@ -789,7 +791,7 @@ public void translateNode( }) .returns( new CoderTypeInformation<>( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( (Coder) VoidCoder.of(), GlobalWindow.Coder.INSTANCE), context.getPipelineOptions())); } else { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java index 952546251c7a..2ccbf24879ae 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java @@ -27,11 +27,12 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; @@ -142,8 +143,8 @@ TypeInformation> getTypeInfo(PCollection collection) { TypeInformation> getTypeInfo( Coder coder, WindowingStrategy windowingStrategy) { - WindowedValue.FullWindowedValueCoder windowedValueCoder = - WindowedValue.getFullCoder(coder, windowingStrategy.getWindowFn().windowCoder()); + WindowedValues.FullWindowedValueCoder windowedValueCoder = + WindowedValues.getFullCoder(coder, windowingStrategy.getWindowFn().windowCoder()); return new CoderTypeInformation<>(windowedValueCoder, options); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingAggregationsTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingAggregationsTranslators.java index 1683ced890c7..60448db365df 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingAggregationsTranslators.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingAggregationsTranslators.java @@ -45,11 +45,12 @@ import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.AppliedCombineFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -210,8 +211,8 @@ WindowDoFnOperator getWindowedAggregateDoFnOperato SingletonKeyedWorkItemCoder.of( keyCoder, inputKvCoder.getValueCoder(), windowingStrategy.getWindowFn().windowCoder()); - WindowedValue.FullWindowedValueCoder> windowedWorkItemCoder = - WindowedValue.getFullCoder(workItemCoder, windowingStrategy.getWindowFn().windowCoder()); + WindowedValues.FullWindowedValueCoder> windowedWorkItemCoder = + WindowedValues.getFullCoder(workItemCoder, windowingStrategy.getWindowFn().windowCoder()); // Key selector WorkItemKeySelector workItemKeySelector = new WorkItemKeySelector<>(keyCoder); @@ -381,17 +382,17 @@ SingleOutputStreamOperator>>> batchGroupByK KvCoder.of(inputKvCoder.getKeyCoder(), accumulatorCoder); Coder>>> windowedAccumCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( accumKvCoder, input.getWindowingStrategy().getWindowFn().windowCoder()); Coder>>>> outputCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(inputKvCoder.getKeyCoder(), IterableCoder.of(accumulatorCoder)), input.getWindowingStrategy().getWindowFn().windowCoder()); TypeInformation>>>> accumulatedTypeInfo = new CoderTypeInformation<>( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( inputKvCoder.getKeyCoder(), IterableCoder.of(IterableCoder.of(inputKvCoder.getValueCoder()))), @@ -468,7 +469,7 @@ SingleOutputStreamOperator>> batchCombinePerKey( accumKvCoder = KvCoder.of(inputKvCoder.getKeyCoder(), accumulatorCoder); windowedAccumCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( accumKvCoder, input.getWindowingStrategy().getWindowFn().windowCoder()); } catch (CannotProvideCoderException e) { throw new RuntimeException(e); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java index 7d83c2e0019f..cad90de8ceed 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java @@ -77,8 +77,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.construction.ModelCoders; import org.apache.beam.sdk.util.construction.NativeTransforms; import org.apache.beam.sdk.util.construction.PTransformTranslation; @@ -97,6 +95,9 @@ import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeDescriptors; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; @@ -326,7 +327,7 @@ private void translateFlatten( }) .returns( new CoderTypeInformation<>( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( (Coder) VoidCoder.of(), GlobalWindow.Coder.INSTANCE), context.getPipelineOptions())); context.addDataStream(Iterables.getOnlyElement(transform.getOutputsMap().values()), result); @@ -427,8 +428,8 @@ private SingleOutputStreamOperator>>> add inputElementCoder.getValueCoder(), windowingStrategy.getWindowFn().windowCoder()); - WindowedValue.FullWindowedValueCoder> windowedWorkItemCoder = - WindowedValue.getFullCoder(workItemCoder, windowingStrategy.getWindowFn().windowCoder()); + WindowedValues.FullWindowedValueCoder> windowedWorkItemCoder = + WindowedValues.getFullCoder(workItemCoder, windowingStrategy.getWindowFn().windowCoder()); WorkItemKeySelector keySelector = new WorkItemKeySelector<>(inputElementCoder.getKeyCoder()); @@ -442,7 +443,7 @@ private SingleOutputStreamOperator>>> add Coder> accumulatorCoder = IterableCoder.of(inputElementCoder.getValueCoder()); Coder>>> outputCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(inputElementCoder.getKeyCoder(), accumulatorCoder), windowingStrategy.getWindowFn().windowCoder()); @@ -520,11 +521,11 @@ private DataStream> translateBoundedSource( BoundedSource boundedSource = (BoundedSource) ReadTranslation.boundedSourceFromProto(payload); @SuppressWarnings("unchecked") - WindowedValue.FullWindowedValueCoder wireCoder = - (WindowedValue.FullWindowedValueCoder) + WindowedValues.FullWindowedValueCoder wireCoder = + (WindowedValues.FullWindowedValueCoder) instantiateCoder(outputCollectionId, pipeline.getComponents()); - WindowedValue.FullWindowedValueCoder sdkCoder = + WindowedValues.FullWindowedValueCoder sdkCoder = getSdkCoder(outputCollectionId, pipeline.getComponents()); CoderTypeInformation> outputTypeInfo = @@ -566,11 +567,11 @@ private static DataStream> translateUnboundedSource( try { @SuppressWarnings("unchecked") - WindowedValue.FullWindowedValueCoder wireCoder = - (WindowedValue.FullWindowedValueCoder) + WindowedValues.FullWindowedValueCoder wireCoder = + (WindowedValues.FullWindowedValueCoder) instantiateCoder(outputCollectionId, pipeline.getComponents()); - WindowedValue.FullWindowedValueCoder sdkCoder = + WindowedValues.FullWindowedValueCoder sdkCoder = getSdkCoder(outputCollectionId, pipeline.getComponents()); CoderTypeInformation> outputTypeInfo = @@ -581,7 +582,7 @@ private static DataStream> translateUnboundedSource( TypeInformation>> withIdTypeInfo = new CoderTypeInformation<>( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( ValueWithRecordId.ValueWithRecordIdCoder.of(sdkCoder.getValueCoder()), windowStrategy.getWindowFn().windowCoder()), pipelineOptions); @@ -625,7 +626,7 @@ private static DataStream> translateUnboundedSource( * @param components the Pipeline components (proto) * @return SDK-side coder for the PCollection */ - private static WindowedValue.FullWindowedValueCoder getSdkCoder( + private static WindowedValues.FullWindowedValueCoder getSdkCoder( String pCollectionId, RunnerApi.Components components) { PipelineNode.PCollectionNode pCollectionNode = @@ -640,8 +641,8 @@ private static WindowedValue.FullWindowedValueCoder getSdkCoder( RehydratedComponents.forComponents(componentsBuilder.build()); try { @SuppressWarnings("unchecked") - WindowedValue.FullWindowedValueCoder res = - (WindowedValue.FullWindowedValueCoder) rehydratedComponents.getCoder(coderId); + WindowedValues.FullWindowedValueCoder res = + (WindowedValues.FullWindowedValueCoder) rehydratedComponents.getCoder(coderId); return res; } catch (IOException ex) { throw new IllegalStateException("Could not get SDK coder.", ex); @@ -677,7 +678,7 @@ private void translateImpulse( TypeInformation> typeInfo = new CoderTypeInformation<>( - WindowedValue.getFullCoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE), context.getPipelineOptions()); long shutdownAfterIdleSourcesMs = context.getPipelineOptions().getShutdownSourcesAfterIdleMs(); @@ -706,7 +707,7 @@ private void translateStreamingImpulse( TypeInformation> typeInfo = new CoderTypeInformation<>( - WindowedValue.getFullCoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE), context.getPipelineOptions()); ObjectMapper objectMapper = new ObjectMapper(); @@ -817,7 +818,7 @@ private void translateExecutableStage( if (stateful || hasSdfProcessFn) { // Stateful/SDF stages are only allowed of KV input. Coder valueCoder = - ((WindowedValue.FullWindowedValueCoder) windowedInputCoder).getValueCoder(); + ((WindowedValues.FullWindowedValueCoder) windowedInputCoder).getValueCoder(); if (!(valueCoder instanceof KvCoder)) { throw new IllegalStateException( String.format( diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java index 36cf035a33be..8a7ec8b59f60 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java @@ -78,7 +78,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.util.construction.ReadTranslation; @@ -93,6 +92,8 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @@ -204,7 +205,7 @@ public void translateNode( TypeInformation>> withIdTypeInfo = new CoderTypeInformation<>( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( ValueWithRecordId.ValueWithRecordIdCoder.of(coder), output.getWindowingStrategy().getWindowFn().windowCoder()), context.getPipelineOptions()); @@ -308,7 +309,7 @@ void translateNode(Impulse transform, FlinkStreamingTranslationContext context) TypeInformation> typeInfo = new CoderTypeInformation<>( - WindowedValue.getFullCoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE), context.getPipelineOptions()); SingleOutputStreamOperator> impulseOperator; @@ -966,7 +967,7 @@ public void translateNode( new KvToFlinkKeyKeySelector<>(inputKvCoder.getKeyCoder()); Coder>>> outputCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( inputKvCoder.getKeyCoder(), IterableCoder.of(inputKvCoder.getValueCoder())), windowingStrategy.getWindowFn().windowCoder()); @@ -1125,8 +1126,8 @@ public void translateNode( inputKvCoder.getValueCoder(), input.getWindowingStrategy().getWindowFn().windowCoder()); - WindowedValue.ValueOnlyWindowedValueCoder> windowedWorkItemCoder = - WindowedValue.getValueOnlyCoder(workItemCoder); + WindowedValues.ValueOnlyWindowedValueCoder> windowedWorkItemCoder = + WindowedValues.getValueOnlyCoder(workItemCoder); CoderTypeInformation>> workItemTypeInfo = new CoderTypeInformation<>(windowedWorkItemCoder, context.getPipelineOptions()); @@ -1179,7 +1180,7 @@ public void flatMap( new SingletonKeyedWorkItem<>( in.getValue().getKey(), in.withValue(in.getValue().getValue())); - out.collect(WindowedValue.valueInGlobalWindow(workItem)); + out.collect(WindowedValues.valueInGlobalWindow(workItem)); } } } @@ -1210,7 +1211,7 @@ public void translateNode( }) .returns( new CoderTypeInformation<>( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( (Coder) VoidCoder.of(), GlobalWindow.Coder.INSTANCE), context.getPipelineOptions())); context.setOutputDataStream(context.getOutput(transform), result); @@ -1311,8 +1312,8 @@ void translateNode(TestStream testStream, FlinkStreamingTranslationContext co } }; - WindowedValue.FullWindowedValueCoder elementCoder = - WindowedValue.getFullCoder(valueCoder, GlobalWindow.Coder.INSTANCE); + WindowedValues.FullWindowedValueCoder elementCoder = + WindowedValues.getFullCoder(valueCoder, GlobalWindow.Coder.INSTANCE); DataStreamSource> source = context @@ -1403,7 +1404,7 @@ private SourceContextWrapper(SourceContext> ctx) { public void collect(WindowedValue> element) { OutputT originalValue = element.getValue().getValue(); WindowedValue output = - WindowedValue.of( + WindowedValues.of( originalValue, element.getTimestamp(), element.getWindows(), element.getPane()); ctx.collect(output); } @@ -1413,7 +1414,7 @@ public void collectWithTimestamp( WindowedValue> element, long timestamp) { OutputT originalValue = element.getValue().getValue(); WindowedValue output = - WindowedValue.of( + WindowedValues.of( originalValue, element.getTimestamp(), element.getWindows(), element.getPane()); ctx.collectWithTimestamp(output, timestamp); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java index 10ba64a77148..24c7b4336e2a 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java @@ -27,13 +27,14 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -112,7 +113,7 @@ public void setCurrentTransform(AppliedPTransform currentTransform) { public Coder> getWindowedInputCoder(PCollection collection) { final Coder valueCoder = collection.getCoder(); - return WindowedValue.getFullCoder( + return WindowedValues.getFullCoder( valueCoder, collection.getWindowingStrategy().getWindowFn().windowCoder()); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/adapter/BeamFlinkDataSetAdapter.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/adapter/BeamFlinkDataSetAdapter.java index 4865a25f70eb..cefc8dd126fc 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/adapter/BeamFlinkDataSetAdapter.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/adapter/BeamFlinkDataSetAdapter.java @@ -32,7 +32,6 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.Preconditions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PipelineOptionsTranslation; import org.apache.beam.sdk.util.construction.graph.PipelineNode; import org.apache.beam.sdk.values.PBegin; @@ -41,6 +40,8 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.flink.api.java.DataSet; @@ -223,11 +224,11 @@ private FlinkBatchPortablePipelineTranslator.PTransformTranslator flink new MapOperator>( flinkInput, BeamAdapterCoderUtils.coderToTypeInformation( - WindowedValue.getValueOnlyCoder( + WindowedValues.getValueOnlyCoder( BeamAdapterCoderUtils.typeInformationToCoder( flinkInput.getType(), coderRegistry)), pipelineOptions), - x -> WindowedValue.valueInGlobalWindow(x), + x -> WindowedValues.valueInGlobalWindow(x), "AddGlobalWindows")); }; } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/adapter/BeamFlinkDataStreamAdapter.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/adapter/BeamFlinkDataStreamAdapter.java index 0a7a1fec803b..435182909fb7 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/adapter/BeamFlinkDataStreamAdapter.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/adapter/BeamFlinkDataStreamAdapter.java @@ -34,7 +34,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.Preconditions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PipelineOptionsTranslation; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; @@ -42,6 +41,8 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.flink.api.common.eventtime.WatermarkStrategy; @@ -239,7 +240,7 @@ public void processElement( Collector> out) throws Exception { out.collect( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( value, ctx.timestamp() == null ? BoundedWindow.TIMESTAMP_MIN_VALUE @@ -247,7 +248,7 @@ public void processElement( } }, BeamAdapterCoderUtils.coderToTypeInformation( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( BeamAdapterCoderUtils.typeInformationToCoder( flinkInput.getType(), coderRegistry), GlobalWindow.Coder.INSTANCE), diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java index 0e437926bd7c..f1ec36564a4b 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java @@ -25,7 +25,7 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.joda.time.Instant; /** diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/AbstractFlinkCombineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/AbstractFlinkCombineRunner.java index c4be4736104f..45ba611f6469 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/AbstractFlinkCombineRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/AbstractFlinkCombineRunner.java @@ -24,8 +24,8 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.flink.util.Collector; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java index eb31fcac6eb1..f153d9144be0 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java @@ -19,7 +19,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.joda.time.Instant; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java index d8c7219bdb63..07a47707827d 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java @@ -20,7 +20,8 @@ import java.util.Collection; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.util.Collector; @@ -43,7 +44,7 @@ public void flatMap(WindowedValue input, Collector> collecto Collection windows = windowFn.assignWindows(new FlinkAssignContext<>(windowFn, input)); for (W window : windows) { collector.collect( - WindowedValue.of(input.getValue(), input.getTimestamp(), window, input.getPane())); + WindowedValues.of(input.getValue(), input.getTimestamp(), window, input.getPane())); } } } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java index f437daf86e71..e12610dfe51c 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java @@ -36,9 +36,10 @@ import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.flink.api.common.functions.AbstractRichFunction; @@ -219,7 +220,7 @@ public void setCollector(Collector> collector) { public void output(TupleTag tag, WindowedValue output) { Objects.requireNonNull(collector) .collect( - WindowedValue.of( + WindowedValues.of( new RawUnionValue(0 /* single output */, output.getValue()), output.getTimestamp(), output.getWindows(), @@ -252,7 +253,7 @@ public void setCollector(Collector> collector) { public void output(TupleTag tag, WindowedValue output) { Objects.requireNonNull(collector) .collect( - WindowedValue.of( + WindowedValues.of( new RawUnionValue(outputMap.get(tag), output.getValue()), output.getTimestamp(), output.getWindows(), diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java index e9d2c3c4e004..6c13f8872e9d 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java @@ -58,10 +58,10 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStagePruningFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStagePruningFunction.java index 467358c154db..6dd4a4135543 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStagePruningFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStagePruningFunction.java @@ -21,7 +21,7 @@ import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.join.RawUnionValue; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExplodeWindowsFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExplodeWindowsFunction.java index b307878fb025..3066b17fcad0 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExplodeWindowsFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExplodeWindowsFunction.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.flink.translation.functions; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.util.Collector; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java index b1b95c6b58e4..c5d6fc3774f7 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java @@ -24,9 +24,9 @@ import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Sessions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.configuration.Configuration; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java index 787b1729cb72..a660f5e9d65d 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java @@ -21,7 +21,7 @@ import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.join.RawUnionValue; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.configuration.Configuration; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNonMergingReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNonMergingReduceFunction.java index fff44e327d02..bcc5a244d3b1 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNonMergingReduceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNonMergingReduceFunction.java @@ -24,8 +24,9 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; @@ -101,7 +102,7 @@ public void reduce( Objects.requireNonNull(wv).getValue().getValue())); } coll.collect( - WindowedValue.of( + WindowedValues.of( KV.of(first.getValue().getKey(), values), combinedTimestamp, first.getWindows(), diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java index 0839812c27b1..31da50372588 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java @@ -24,9 +24,9 @@ import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Sessions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.common.functions.RichGroupCombineFunction; import org.apache.flink.configuration.Configuration; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java index 13998693de7f..bc1cf36639a0 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java @@ -24,9 +24,9 @@ import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Sessions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.configuration.Configuration; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java index 16eac410f278..95fc4e8ab8b6 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java @@ -49,10 +49,10 @@ import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.flink.api.common.functions.RichGroupReduceFunction; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/HashingFlinkCombineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/HashingFlinkCombineRunner.java index 455a8540e505..19c6a6324bf0 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/HashingFlinkCombineRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/HashingFlinkCombineRunner.java @@ -32,8 +32,9 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.flink.api.java.tuple.Tuple2; @@ -115,7 +116,7 @@ public void combine( AccumT accumulator = entry.getValue().f0; Instant windowTimestamp = entry.getValue().f1; out.collect( - WindowedValue.of( + WindowedValues.of( KV.of( key, flinkCombiner.extractOutput( diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunction.java index 577949bcbc8d..779c47dd0e6f 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunction.java @@ -17,7 +17,8 @@ */ package org.apache.beam.runners.flink.translation.functions; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; @@ -58,7 +59,7 @@ public void run(SourceContext> sourceContext) throws Excep if (Iterables.isEmpty(impulseEmitted.get())) { synchronized (sourceContext.getCheckpointLock()) { // emit single impulse element - sourceContext.collect(WindowedValue.valueInGlobalWindow(new byte[0])); + sourceContext.collect(WindowedValues.valueInGlobalWindow(new byte[0])); impulseEmitted.add(true); } } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java index d44428a84f51..200403a81e95 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java @@ -33,8 +33,8 @@ import org.apache.beam.sdk.transforms.Materializations.MultimapView; import org.apache.beam.sdk.transforms.ViewFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.flink.api.common.functions.BroadcastVariableInitializer; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SingleWindowFlinkCombineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SingleWindowFlinkCombineRunner.java index 58ecfdf96e80..f33ecda636be 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SingleWindowFlinkCombineRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SingleWindowFlinkCombineRunner.java @@ -24,8 +24,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -97,7 +98,7 @@ public void combine( final AccumT accumulator = combinedState.f0; final Instant windowTimestamp = combinedState.f1; out.collect( - WindowedValue.of( + WindowedValues.of( KV.of( key, flinkCombiner.extractOutput( diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java index 149c3e284032..976b3dfe0868 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java @@ -26,8 +26,9 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -109,7 +110,7 @@ public void combine( } else { // emit the value that we currently have out.collect( - WindowedValue.of( + WindowedValues.of( KV.of( key, flinkCombiner.extractOutput( @@ -130,7 +131,7 @@ public void combine( // emit the final accumulator out.collect( - WindowedValue.of( + WindowedValues.of( KV.of( key, flinkCombiner.extractOutput( @@ -161,7 +162,7 @@ private void mergeWindow(List>> elements) { WindowedValue> value = elements.get(j); elements.set( j, - WindowedValue.of( + WindowedValues.of( value.getValue(), value.getTimestamp(), currentWindow, value.getPane())); } currentStart = i; @@ -174,7 +175,7 @@ private void mergeWindow(List>> elements) { WindowedValue> value = elements.get(j); elements.set( j, - WindowedValue.of( + WindowedValues.of( value.getValue(), value.getTimestamp(), currentWindow, value.getPane())); } } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java index 7e879fb27ddf..30bb94f1d1d5 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java @@ -19,8 +19,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/WindowedKvKeySelector.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/WindowedKvKeySelector.java index 13f55a4c1456..22766ea0c3e2 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/WindowedKvKeySelector.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/WindowedKvKeySelector.java @@ -20,8 +20,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.primitives.Bytes; import org.apache.flink.api.common.typeinfo.TypeInformation; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/ImpulseInputFormat.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/ImpulseInputFormat.java index 153c875ca143..2a0cf06cc3f2 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/ImpulseInputFormat.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/ImpulseInputFormat.java @@ -19,7 +19,8 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.flink.api.common.io.DefaultInputSplitAssigner; import org.apache.flink.api.common.io.RichInputFormat; import org.apache.flink.api.common.io.statistics.BaseStatistics; @@ -88,7 +89,7 @@ public WindowedValue nextRecord(WindowedValue windowedValue) { if (windowedValue != null) { return windowedValue; } - return WindowedValue.valueInGlobalWindow(new byte[0]); + return WindowedValues.valueInGlobalWindow(new byte[0]); } @Override diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java index a1b8bced7a1d..8a8d4cf3e9a5 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java @@ -29,7 +29,8 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.flink.api.common.io.DefaultInputSplitAssigner; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.io.RichInputFormat; @@ -162,7 +163,7 @@ public WindowedValue nextRecord(WindowedValue t) throws IOException { final Instant timestamp = reader.getCurrentTimestamp(); // advance reader to have a record ready next time inputAvailable = readerInvoker.invokeAdvance(reader); - return WindowedValue.of(current, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); + return WindowedValues.of(current, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); } return null; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index 01b12cfa717a..701df1c14267 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -84,10 +84,10 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.NoopLock; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java index b48d6f2e4ec4..d5a7ff035efc 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java @@ -97,7 +97,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; @@ -105,6 +104,8 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.StatusRuntimeException; @@ -530,7 +531,7 @@ void setTimer(Timer timerElement, TimerInternals.TimerData timerData) { FlinkKey encodedKey = (FlinkKey) keySelector.getKey( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( (InputT) KV.of(timerElement.getUserKey(), null))); // We have to synchronize to ensure the state backend is not concurrently accessed by the // state requests @@ -562,7 +563,8 @@ class SdfFlinkTimerInternalsFactory implements TimerInternalsFactory { @Override public TimerInternals timerInternalsForKey(InputT key) { try { - FlinkKey encodedKey = (FlinkKey) keySelector.getKey(WindowedValue.valueInGlobalWindow(key)); + FlinkKey encodedKey = + (FlinkKey) keySelector.getKey(WindowedValues.valueInGlobalWindow(key)); return new SdfFlinkTimerInternals(encodedKey); } catch (Exception e) { throw new RuntimeException("Couldn't get a timer internals", e); @@ -658,7 +660,8 @@ class SdfFlinkStateInternalsFactory implements StateInternalsFactory { @Override public StateInternals stateInternalsForKey(InputT key) { try { - FlinkKey encodedKey = (FlinkKey) keySelector.getKey(WindowedValue.valueInGlobalWindow(key)); + FlinkKey encodedKey = + (FlinkKey) keySelector.getKey(WindowedValues.valueInGlobalWindow(key)); return new SdfFlinkStateInternals(encodedKey); } catch (Exception e) { throw new RuntimeException("Couldn't get a state internals", e); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KvToFlinkKeyKeySelector.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KvToFlinkKeyKeySelector.java index a852a724c040..76c5e7ece6ed 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KvToFlinkKeyKeySelector.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KvToFlinkKeyKeySelector.java @@ -19,8 +19,8 @@ import org.apache.beam.runners.flink.adapter.FlinkKey; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/PartialReduceBundleOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/PartialReduceBundleOperator.java index 03570143231b..c8f36e234415 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/PartialReduceBundleOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/PartialReduceBundleOperator.java @@ -34,10 +34,11 @@ import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Sessions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ArrayListMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Multimap; @@ -173,7 +174,7 @@ protected DoFn, KV> getDoFn() { @ProcessElement public void processElement(ProcessContext c, BoundedWindow window) throws Exception { WindowedValue> windowedValue = - WindowedValue.of(c.element(), c.timestamp(), window, c.pane()); + WindowedValues.of(c.element(), c.timestamp(), window, c.pane()); state.put(Objects.requireNonNull(c.element()).getKey(), windowedValue); } }; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SdfFlinkKeyKeySelector.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SdfFlinkKeyKeySelector.java index b316726e74f8..1e9d27db21e9 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SdfFlinkKeyKeySelector.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SdfFlinkKeyKeySelector.java @@ -19,8 +19,8 @@ import org.apache.beam.runners.flink.adapter.FlinkKey; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java index 6f2f473feddc..9284a7ae99a3 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java @@ -20,7 +20,7 @@ import java.util.Collections; import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** Singleton keyed word item. */ public class SingletonKeyedWorkItem implements KeyedWorkItem { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java index 747bd9f71894..424782650f3a 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java @@ -27,7 +27,8 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StructuredCoder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** Singleton keyed work item coder. */ @@ -46,14 +47,14 @@ public static SingletonKeyedWorkItemCoder of( private final Coder keyCoder; private final Coder elemCoder; private final Coder windowCoder; - private final WindowedValue.FullWindowedValueCoder valueCoder; + private final WindowedValues.FullWindowedValueCoder valueCoder; private SingletonKeyedWorkItemCoder( Coder keyCoder, Coder elemCoder, Coder windowCoder) { this.keyCoder = keyCoder; this.elemCoder = elemCoder; this.windowCoder = windowCoder; - valueCoder = WindowedValue.FullWindowedValueCoder.of(elemCoder, windowCoder); + valueCoder = WindowedValues.FullWindowedValueCoder.of(elemCoder, windowCoder); } public Coder getKeyCoder() { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java index d80dd60a5925..3f8758f610b1 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java @@ -44,10 +44,11 @@ import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.flink.api.java.functions.KeySelector; @@ -152,7 +153,7 @@ public void outputWindowedValue( Collection windows, PaneInfo pane) { outputManager.output( - mainOutputTag, WindowedValue.of(output, timestamp, windows, pane)); + mainOutputTag, WindowedValues.of(output, timestamp, windows, pane)); } @Override @@ -162,7 +163,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - outputManager.output(tag, WindowedValue.of(output, timestamp, windows, pane)); + outputManager.output(tag, WindowedValues.of(output, timestamp, windows, pane)); } }, sideInputReader, @@ -181,7 +182,7 @@ protected void fireTimer(TimerInternals.TimerData timer) { return; } doFnRunner.processElement( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( KeyedWorkItems.timersWorkItem( (byte[]) keyedStateInternals.getKey(), Collections.singletonList(timer)))); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java index 60b20f375f22..fad97e2d7dfd 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java @@ -39,10 +39,11 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.java.functions.KeySelector; @@ -149,7 +150,7 @@ protected DoFn, KV> getDoFn() { protected void fireTimer(TimerData timer) { timerInternals.onFiredOrDeletedTimer(timer); doFnRunner.processElement( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( KeyedWorkItems.timersWorkItem( (K) keyedStateInternals.getKey(), Collections.singletonList(timer)))); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java index d809f4287983..36bdfd596eed 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java @@ -21,7 +21,7 @@ import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.flink.adapter.FlinkKey; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/DedupingOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/DedupingOperator.java index 9d238aa36110..f45632a566cb 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/DedupingOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/DedupingOperator.java @@ -21,8 +21,8 @@ import org.apache.beam.runners.flink.adapter.FlinkKey; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeutils.base.LongSerializer; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/StreamingImpulseSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/StreamingImpulseSource.java index 871d7a5a3989..e8b6ee786152 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/StreamingImpulseSource.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/StreamingImpulseSource.java @@ -18,7 +18,8 @@ package org.apache.beam.runners.flink.translation.wrappers.streaming.io; import java.nio.charset.StandardCharsets; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,7 +61,7 @@ public void run(SourceContext> ctx) { while (running && (messageCount == 0 || count < subtaskCount)) { synchronized (ctx.getCheckpointLock()) { ctx.collect( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( String.valueOf(count).getBytes(StandardCharsets.UTF_8))); count++; } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/TestStreamSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/TestStreamSource.java index bacba9402459..52b81b8f9a43 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/TestStreamSource.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/TestStreamSource.java @@ -23,8 +23,9 @@ import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; import org.apache.flink.streaming.api.watermark.Watermark; import org.joda.time.Instant; @@ -56,7 +57,7 @@ public void run(SourceContext> ctx) throws CoderException { for (TimestampedValue element : ((TestStream.ElementEvent) event).getElements()) { Instant timestamp = element.getTimestamp(); WindowedValue value = - WindowedValue.of( + WindowedValues.of( element.getValue(), timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); ctx.collectWithTimestamp(value, timestamp.getMillis()); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java index d6beafad8d93..7811f1f85a67 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java @@ -36,11 +36,12 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.UnboundedReadFromBoundedSource; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.operators.ProcessingTimeService.ProcessingTimeCallback; @@ -341,7 +342,7 @@ private void emitElement( Instant timestamp = reader.getCurrentTimestamp(); WindowedValue> windowedValue = - WindowedValue.of( + WindowedValues.of( new ValueWithRecordId<>(item, recordId), timestamp, GlobalWindow.INSTANCE, diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSource.java index 685bb870c5fd..0667a37789fc 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSource.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSource.java @@ -22,7 +22,7 @@ import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSource; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSourceSplit; import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.SourceReader; import org.apache.flink.api.connector.source.SourceReaderContext; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReader.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReader.java index 6b23dd13c9b8..05872accc36a 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReader.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReader.java @@ -32,7 +32,8 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.flink.api.common.eventtime.Watermark; import org.apache.flink.api.connector.source.ReaderOutput; @@ -125,7 +126,7 @@ public InputStatus pollNext(ReaderOutput> output) throws Except final @Nonnull Source.Reader splitReader = currentReader; T record = splitReader.getCurrent(); WindowedValue windowedValue = - WindowedValue.of( + WindowedValues.of( record, splitReader.getCurrentTimestamp(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); if (timestampExtractor == null) { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSource.java index 8ef2edfa606e..5d545fd53a1e 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSource.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSource.java @@ -22,8 +22,8 @@ import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSource; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSourceSplit; import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.SourceReader; import org.apache.flink.api.connector.source.SourceReaderContext; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java index 54c23bea3fb1..8837f3b9ec4e 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java @@ -40,8 +40,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.flink.api.common.eventtime.Watermark; import org.apache.flink.api.connector.source.ReaderOutput; @@ -254,7 +255,7 @@ private void emitRecord( Instant timestamp = reader.getCurrentTimestamp(); WindowedValue> windowedValue = - WindowedValue.of( + WindowedValues.of( new ValueWithRecordId<>(item, recordId), timestamp, GlobalWindow.INSTANCE, diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElements.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElements.java index 91887dd04ed7..0c2ba87c4baa 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElements.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElements.java @@ -28,7 +28,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java index 312f8812ff1c..10a3182f90f4 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java @@ -32,7 +32,7 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.flink.api.common.state.ListState; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java index 5d08beb938fd..dc07caf5bfd7 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java @@ -36,8 +36,9 @@ import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.ExecutionMode; @@ -112,7 +113,7 @@ public void testDefaults() { @Test(expected = Exception.class) public void parDoBaseClassPipelineOptionsNullTest() { TupleTag mainTag = new TupleTag<>("main-output"); - Coder> coder = WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + Coder> coder = WindowedValues.getValueOnlyCoder(StringUtf8Coder.of()); new DoFnOperator<>( new TestDoFn(), "stepName", @@ -138,7 +139,7 @@ public void parDoBaseClassPipelineOptionsSerializationTest() throws Exception { TupleTag mainTag = new TupleTag<>("main-output"); - Coder> coder = WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + Coder> coder = WindowedValues.getValueOnlyCoder(StringUtf8Coder.of()); DoFnOperator doFnOperator = new DoFnOperator<>( new TestDoFn(), @@ -174,7 +175,7 @@ mainTag, coder, new SerializablePipelineOptions(FlinkPipelineOptions.defaults()) // execute once to access options testHarness.processElement( new StreamRecord<>( - WindowedValue.of( + WindowedValues.of( new Object(), Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING))); testHarness.close(); diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/BoundedSourceRestoreTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/BoundedSourceRestoreTest.java index a7183df67b44..c785d02fad1f 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/BoundedSourceRestoreTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/BoundedSourceRestoreTest.java @@ -30,10 +30,10 @@ import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter; import org.apache.beam.sdk.util.construction.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter.Checkpoint; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.operators.StreamSource; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java index 0435c70c7672..f4e7901f2f39 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java @@ -48,9 +48,10 @@ import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.join.RawUnionValue; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.Struct; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.flink.api.common.cache.DistributedCache; @@ -182,9 +183,9 @@ public void expectedInputsAreSent() throws Exception { FnDataReceiver> receiver = Mockito.mock(FnDataReceiver.class); when(bundle.getInputReceivers()).thenReturn(ImmutableMap.of("input", receiver)); - WindowedValue one = WindowedValue.valueInGlobalWindow(1); - WindowedValue two = WindowedValue.valueInGlobalWindow(2); - WindowedValue three = WindowedValue.valueInGlobalWindow(3); + WindowedValue one = WindowedValues.valueInGlobalWindow(1); + WindowedValue two = WindowedValues.valueInGlobalWindow(2); + WindowedValue three = WindowedValues.valueInGlobalWindow(3); function.mapPartition(Arrays.asList(one, two, three), collector); verify(receiver).accept(one); @@ -195,9 +196,9 @@ public void expectedInputsAreSent() throws Exception { @Test public void outputsAreTaggedCorrectly() throws Exception { - WindowedValue three = WindowedValue.valueInGlobalWindow(3); - WindowedValue four = WindowedValue.valueInGlobalWindow(4); - WindowedValue five = WindowedValue.valueInGlobalWindow(5); + WindowedValue three = WindowedValues.valueInGlobalWindow(3); + WindowedValue four = WindowedValues.valueInGlobalWindow(4); + WindowedValue five = WindowedValues.valueInGlobalWindow(5); Map outputTagMap = ImmutableMap.of( "one", 1, diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunctionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunctionTest.java index e29cd8d6ad5c..554c4207b4a5 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunctionTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunctionTest.java @@ -28,7 +28,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.OperatorStateStore; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DedupingOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DedupingOperatorTest.java index 9576baa4c66b..4b0a38a60615 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DedupingOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DedupingOperatorTest.java @@ -25,8 +25,9 @@ import java.nio.charset.StandardCharsets; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.DedupingOperator; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -53,22 +54,23 @@ public void testDeduping() throws Exception { harness.processElement( new StreamRecord<>( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( new ValueWithRecordId<>(key1, key1.getBytes(StandardCharsets.UTF_8))))); harness.processElement( new StreamRecord<>( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( new ValueWithRecordId<>(key2, key2.getBytes(StandardCharsets.UTF_8))))); harness.processElement( new StreamRecord<>( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( new ValueWithRecordId<>(key1, key1.getBytes(StandardCharsets.UTF_8))))); assertThat( stripStreamRecordFromWindowedValue(harness.getOutput()), - contains(WindowedValue.valueInGlobalWindow(key1), WindowedValue.valueInGlobalWindow(key2))); + contains( + WindowedValues.valueInGlobalWindow(key1), WindowedValues.valueInGlobalWindow(key2))); OperatorSubtaskState snapshot = harness.snapshot(0L, 0L); @@ -83,17 +85,17 @@ public void testDeduping() throws Exception { harness.processElement( new StreamRecord<>( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( new ValueWithRecordId<>(key2, key2.getBytes(StandardCharsets.UTF_8))))); harness.processElement( new StreamRecord<>( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( new ValueWithRecordId<>(key3, key3.getBytes(StandardCharsets.UTF_8))))); assertThat( stripStreamRecordFromWindowedValue(harness.getOutput()), - contains(WindowedValue.valueInGlobalWindow(key3))); + contains(WindowedValues.valueInGlobalWindow(key3))); harness.close(); } diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java index f0d8816bdeab..c2556d7229ba 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java @@ -81,12 +81,13 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Function; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.FluentIterable; @@ -145,7 +146,7 @@ public void setUp() { @Test public void testSingleOutput() throws Exception { - Coder> coder = WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + Coder> coder = WindowedValues.getValueOnlyCoder(StringUtf8Coder.of()); TupleTag outputTag = new TupleTag<>("main-output"); @@ -173,11 +174,11 @@ outputTag, coder, new SerializablePipelineOptions(FlinkPipelineOptions.defaults( testHarness.open(); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("Hello"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("Hello"))); assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), - contains(WindowedValue.valueInGlobalWindow("Hello"))); + contains(WindowedValues.valueInGlobalWindow("Hello"))); testHarness.close(); } @@ -186,8 +187,8 @@ outputTag, coder, new SerializablePipelineOptions(FlinkPipelineOptions.defaults( @SuppressWarnings("unchecked") public void testMultiOutputOutput() throws Exception { - WindowedValue.ValueOnlyWindowedValueCoder coder = - WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + WindowedValues.ValueOnlyWindowedValueCoder coder = + WindowedValues.getValueOnlyCoder(StringUtf8Coder.of()); TupleTag mainOutput = new TupleTag<>("main-output"); TupleTag additionalOutput1 = new TupleTag<>("output-1"); @@ -239,25 +240,25 @@ public void testMultiOutputOutput() throws Exception { testHarness.open(); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("one"))); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("two"))); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("hello"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("one"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("two"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("hello"))); assertThat( this.stripStreamRecord(testHarness.getOutput()), - contains(WindowedValue.valueInGlobalWindow("got: hello"))); + contains(WindowedValues.valueInGlobalWindow("got: hello"))); assertThat( this.stripStreamRecord(testHarness.getSideOutput(tagsToOutputTags.get(additionalOutput1))), contains( - WindowedValue.valueInGlobalWindow("extra: one"), - WindowedValue.valueInGlobalWindow("got: hello"))); + WindowedValues.valueInGlobalWindow("extra: one"), + WindowedValues.valueInGlobalWindow("got: hello"))); assertThat( this.stripStreamRecord(testHarness.getSideOutput(tagsToOutputTags.get(additionalOutput2))), contains( - WindowedValue.valueInGlobalWindow("extra: two"), - WindowedValue.valueInGlobalWindow("got: hello"))); + WindowedValues.valueInGlobalWindow("extra: two"), + WindowedValues.valueInGlobalWindow("got: hello"))); testHarness.close(); } @@ -341,11 +342,11 @@ public void onProcessingTime(OnTimerContext context) { }; VarIntCoder keyCoder = VarIntCoder.of(); - WindowedValue.FullWindowedValueCoder inputCoder = - WindowedValue.getFullCoder(keyCoder, windowingStrategy.getWindowFn().windowCoder()); + WindowedValues.FullWindowedValueCoder inputCoder = + WindowedValues.getFullCoder(keyCoder, windowingStrategy.getWindowFn().windowCoder()); - WindowedValue.FullWindowedValueCoder outputCoder = - WindowedValue.getFullCoder( + WindowedValues.FullWindowedValueCoder outputCoder = + WindowedValues.getFullCoder( StringUtf8Coder.of(), windowingStrategy.getWindowFn().windowCoder()); KeySelector, FlinkKey> keySelector = @@ -391,7 +392,7 @@ public void onProcessingTime(OnTimerContext context) { // this should register the two timers above testHarness.processElement( - new StreamRecord<>(WindowedValue.of(13, new Instant(0), window1, PaneInfo.NO_FIRING))); + new StreamRecord<>(WindowedValues.of(13, new Instant(0), window1, PaneInfo.NO_FIRING))); assertThat(stripStreamRecordFromWindowedValue(testHarness.getOutput()), emptyIterable()); @@ -415,14 +416,15 @@ public void onProcessingTime(OnTimerContext context) { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), containsInAnyOrder( - WindowedValue.of( + WindowedValues.of( eventTimeMessage + eventTimerId, timerTimestamp, window1, PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( eventTimeMessage + eventTimerId2, timerTimestamp.minus(Duration.millis(1)), window1, PaneInfo.NO_FIRING), - WindowedValue.of(processingTimeMessage, new Instant(10), window1, PaneInfo.NO_FIRING))); + WindowedValues.of( + processingTimeMessage, new Instant(10), window1, PaneInfo.NO_FIRING))); testHarness.close(); } @@ -431,7 +433,7 @@ public void onProcessingTime(OnTimerContext context) { public void testWatermarkUpdateAfterWatermarkHoldRelease() throws Exception { Coder>> coder = - WindowedValue.getValueOnlyCoder(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())); + WindowedValues.getValueOnlyCoder(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())); TupleTag> outputTag = new TupleTag<>("main-output"); List emittedWatermarkHolds = new ArrayList<>(); @@ -553,7 +555,7 @@ void emitWatermarkIfHoldChanged(long currentWatermarkHold) { // process first element, set hold to `now', setup timer for `now + 1' testHarness.processElement( new StreamRecord<>( - WindowedValue.timestampedValueInGlobalWindow(KV.of("Key", "Hello"), now))); + WindowedValues.timestampedValueInGlobalWindow(KV.of("Key", "Hello"), now))); assertThat(emittedWatermarkHolds, is(equalTo(Collections.singletonList(now.getMillis())))); @@ -566,7 +568,7 @@ void emitWatermarkIfHoldChanged(long currentWatermarkHold) { // process second element, verify we emitted changed hold testHarness.processElement( new StreamRecord<>( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of("Key", "Hello"), now.plus(Duration.millis(2))))); assertThat( @@ -601,9 +603,9 @@ public void processElement(ProcessContext context) { VarIntCoder keyCoder = VarIntCoder.of(); Coder> inputCoder = - WindowedValue.getFullCoder(keyCoder, windowingStrategy.getWindowFn().windowCoder()); + WindowedValues.getFullCoder(keyCoder, windowingStrategy.getWindowFn().windowCoder()); Coder> outputCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( StringUtf8Coder.of(), windowingStrategy.getWindowFn().windowCoder()); KeySelector, FlinkKey> keySelector = @@ -644,11 +646,11 @@ public void processElement(ProcessContext context) { // this should not be late testHarness.processElement( - new StreamRecord<>(WindowedValue.of(13, new Instant(0), window1, PaneInfo.NO_FIRING))); + new StreamRecord<>(WindowedValues.of(13, new Instant(0), window1, PaneInfo.NO_FIRING))); assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), - contains(WindowedValue.of("13", new Instant(0), window1, PaneInfo.NO_FIRING))); + contains(WindowedValues.of("13", new Instant(0), window1, PaneInfo.NO_FIRING))); testHarness.getOutput().clear(); @@ -656,11 +658,11 @@ public void processElement(ProcessContext context) { // this should still not be considered late testHarness.processElement( - new StreamRecord<>(WindowedValue.of(17, new Instant(0), window1, PaneInfo.NO_FIRING))); + new StreamRecord<>(WindowedValues.of(17, new Instant(0), window1, PaneInfo.NO_FIRING))); assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), - contains(WindowedValue.of("17", new Instant(0), window1, PaneInfo.NO_FIRING))); + contains(WindowedValues.of("17", new Instant(0), window1, PaneInfo.NO_FIRING))); testHarness.getOutput().clear(); @@ -668,7 +670,7 @@ public void processElement(ProcessContext context) { // this should now be considered late testHarness.processElement( - new StreamRecord<>(WindowedValue.of(17, new Instant(0), window1, PaneInfo.NO_FIRING))); + new StreamRecord<>(WindowedValues.of(17, new Instant(0), window1, PaneInfo.NO_FIRING))); assertThat(stripStreamRecordFromWindowedValue(testHarness.getOutput()), emptyIterable()); @@ -703,18 +705,18 @@ public void testStateGCForStatefulFn() throws Exception { testHarness.processElement( new StreamRecord<>( - WindowedValue.of(KV.of("key1", 5), new Instant(1), window1, PaneInfo.NO_FIRING))); + WindowedValues.of(KV.of("key1", 5), new Instant(1), window1, PaneInfo.NO_FIRING))); testHarness.processElement( new StreamRecord<>( - WindowedValue.of(KV.of("key2", 7), new Instant(3), window1, PaneInfo.NO_FIRING))); + WindowedValues.of(KV.of("key2", 7), new Instant(3), window1, PaneInfo.NO_FIRING))); assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( - WindowedValue.of( + WindowedValues.of( KV.of("key1", 5 + offset), new Instant(1), window1, PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( KV.of("key2", 7 + offset), new Instant(3), window1, PaneInfo.NO_FIRING))); // 2 entries for the elements and 2 for the pending timers @@ -737,9 +739,9 @@ public void testStateGCForStatefulFn() throws Exception { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( - WindowedValue.of( + WindowedValues.of( KV.of("key1", timerOutput), new Instant(9), window1, PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( KV.of("key2", timerOutput), new Instant(9), window1, PaneInfo.NO_FIRING))); testHarness.close(); @@ -763,11 +765,11 @@ public void testGCForGlobalWindow() throws Exception { // Check global window cleanup via final watermark, _without_ cleanup timers testHarness.processElement( new StreamRecord<>( - WindowedValue.of( + WindowedValues.of( KV.of("key1", 5), new Instant(23), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING))); testHarness.processElement( new StreamRecord<>( - WindowedValue.of( + WindowedValues.of( KV.of("key2", 6), new Instant(42), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING))); // timers set by the transform @@ -800,7 +802,7 @@ public void testGCForGlobalWindow() throws Exception { // Any new state will also be cleaned up on close testHarness.processElement( new StreamRecord<>( - WindowedValue.of( + WindowedValues.of( KV.of("key2", 6), new Instant(42), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING))); // Close sends Flink's max watermark and will cleanup again @@ -848,8 +850,8 @@ public void onTimer(OnTimerContext context, @StateId(stateId) ValueState } }; - WindowedValue.FullWindowedValueCoder> coder = - WindowedValue.getFullCoder( + WindowedValues.FullWindowedValueCoder> coder = + WindowedValues.getFullCoder( KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()), windowingStrategy.getWindowFn().windowCoder()); @@ -896,7 +898,7 @@ public void testKeyedParDoSideInputs() throws Exception { void testSideInputs(boolean keyed) throws Exception { - Coder> coder = WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + Coder> coder = WindowedValues.getValueOnlyCoder(StringUtf8Coder.of()); TupleTag outputTag = new TupleTag<>("main-output"); @@ -1029,7 +1031,7 @@ public void processElement( KvCoder coder = KvCoder.of(keyCoder, VarLongCoder.of()); FullWindowedValueCoder> kvCoder = - WindowedValue.getFullCoder(coder, windowingStrategy.getWindowFn().windowCoder()); + WindowedValues.getFullCoder(coder, windowingStrategy.getWindowFn().windowCoder()); TypeInformation keyCoderInfo = ValueTypeInfo.of(FlinkKey.class); @@ -1048,9 +1050,9 @@ public void processElement( testHarness.open(); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("a", 100L)))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("a", 100L)))); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("a", 100L)))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("a", 100L)))); OperatorSubtaskState snapshot = testHarness.snapshot(0, 0); testHarness.close(); @@ -1070,19 +1072,19 @@ public void processElement( // after restore: counter = 2 testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("a", 100L)))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("a", 100L)))); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("a", 4L)))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("a", 4L)))); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("a", 5L)))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("a", 5L)))); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("a", 100L)))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("a", 100L)))); assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow(KV.of("a", 4L)), - WindowedValue.valueInGlobalWindow(KV.of("a", 5L)))); + WindowedValues.valueInGlobalWindow(KV.of("a", 4L)), + WindowedValues.valueInGlobalWindow(KV.of("a", 5L)))); testHarness.close(); } @@ -1092,7 +1094,7 @@ public void nonKeyedParDoSideInputCheckpointing() throws Exception { sideInputCheckpointing( () -> { Coder> coder = - WindowedValue.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()); + WindowedValues.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()); TupleTag outputTag = new TupleTag<>("main-output"); ImmutableMap> sideInputMapping = @@ -1132,7 +1134,7 @@ public void keyedParDoSideInputCheckpointing() throws Exception { () -> { StringUtf8Coder keyCoder = StringUtf8Coder.of(); Coder> coder = - WindowedValue.getFullCoder(keyCoder, IntervalWindow.getCoder()); + WindowedValues.getFullCoder(keyCoder, IntervalWindow.getCoder()); TupleTag outputTag = new TupleTag<>("main-output"); KeySelector, FlinkKey> keySelector = @@ -1236,7 +1238,7 @@ public void nonKeyedParDoPushbackDataCheckpointing() throws Exception { pushbackDataCheckpointing( () -> { Coder> coder = - WindowedValue.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()); + WindowedValues.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()); TupleTag outputTag = new TupleTag<>("main-output"); @@ -1277,7 +1279,7 @@ public void keyedParDoPushbackDataCheckpointing() throws Exception { () -> { StringUtf8Coder keyCoder = StringUtf8Coder.of(); Coder> coder = - WindowedValue.getFullCoder(keyCoder, IntervalWindow.getCoder()); + WindowedValues.getFullCoder(keyCoder, IntervalWindow.getCoder()); TupleTag outputTag = new TupleTag<>("main-output"); @@ -1406,11 +1408,11 @@ public void onEventTime(OnTimerContext context) { }; VarIntCoder keyCoder = VarIntCoder.of(); - WindowedValue.FullWindowedValueCoder inputCoder = - WindowedValue.getFullCoder(keyCoder, windowingStrategy.getWindowFn().windowCoder()); + WindowedValues.FullWindowedValueCoder inputCoder = + WindowedValues.getFullCoder(keyCoder, windowingStrategy.getWindowFn().windowCoder()); - WindowedValue.FullWindowedValueCoder outputCoder = - WindowedValue.getFullCoder( + WindowedValues.FullWindowedValueCoder outputCoder = + WindowedValues.getFullCoder( StringUtf8Coder.of(), windowingStrategy.getWindowFn().windowCoder()); TupleTag outputTag = new TupleTag<>("main-output"); @@ -1444,7 +1446,7 @@ public void onEventTime(OnTimerContext context) { // this should register a timer testHarness.processElement( - new StreamRecord<>(WindowedValue.of(13, new Instant(0), window1, PaneInfo.NO_FIRING))); + new StreamRecord<>(WindowedValues.of(13, new Instant(0), window1, PaneInfo.NO_FIRING))); assertThat(stripStreamRecordFromWindowedValue(testHarness.getOutput()), emptyIterable()); @@ -1471,7 +1473,7 @@ public void onEventTime(OnTimerContext context) { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), - contains(WindowedValue.of(outputMessage, timerTimestamp, window1, PaneInfo.NO_FIRING))); + contains(WindowedValues.of(outputMessage, timerTimestamp, window1, PaneInfo.NO_FIRING))); testHarness.close(); } @@ -1514,8 +1516,8 @@ OneInputStreamOperatorTestHarness, WindowedValue> creat @Test public void testBundle() throws Exception { - WindowedValue.ValueOnlyWindowedValueCoder windowedValueCoder = - WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + WindowedValues.ValueOnlyWindowedValueCoder windowedValueCoder = + WindowedValues.getValueOnlyCoder(StringUtf8Coder.of()); TupleTag outputTag = new TupleTag<>("main-output"); FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); @@ -1535,7 +1537,7 @@ public void finishBundle(FinishBundleContext context) { DoFnOperator.MultiOutputOutputManagerFactory outputManagerFactory = new DoFnOperator.MultiOutputOutputManagerFactory<>( outputTag, - WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), new SerializablePipelineOptions(options)); DoFnOperator doFnOperator = @@ -1561,17 +1563,17 @@ public void finishBundle(FinishBundleContext context) { testHarness.open(); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("a"))); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("b"))); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("c"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("a"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("b"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("c"))); assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow("a"), - WindowedValue.valueInGlobalWindow("b"), - WindowedValue.valueInGlobalWindow("finishBundle"), - WindowedValue.valueInGlobalWindow("c"))); + WindowedValues.valueInGlobalWindow("a"), + WindowedValues.valueInGlobalWindow("b"), + WindowedValues.valueInGlobalWindow("finishBundle"), + WindowedValues.valueInGlobalWindow("c"))); // draw a snapshot OperatorSubtaskState snapshot = testHarness.snapshot(0, 0); @@ -1583,7 +1585,7 @@ public void finishBundle(FinishBundleContext context) { List>> bufferedElements = pushedBackElementsHandler.getElements().collect(Collectors.toList()); assertThat( - bufferedElements, contains(KV.of(0, WindowedValue.valueInGlobalWindow("finishBundle")))); + bufferedElements, contains(KV.of(0, WindowedValues.valueInGlobalWindow("finishBundle")))); testHarness.close(); @@ -1614,7 +1616,7 @@ public void finishBundle(FinishBundleContext context) { newHarness.open(); // startBundle will output the buffered elements. - newHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("d"))); + newHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("d"))); // check finishBundle by timeout newHarness.setProcessingTime(10); @@ -1622,9 +1624,9 @@ public void finishBundle(FinishBundleContext context) { assertThat( stripStreamRecordFromWindowedValue(newHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow("finishBundle"), - WindowedValue.valueInGlobalWindow("d"), - WindowedValue.valueInGlobalWindow("finishBundle"))); + WindowedValues.valueInGlobalWindow("finishBundle"), + WindowedValues.valueInGlobalWindow("d"), + WindowedValues.valueInGlobalWindow("finishBundle"))); // No bundle will be created when sending the MAX watermark // (unless pushed back items are emitted) @@ -1633,9 +1635,9 @@ public void finishBundle(FinishBundleContext context) { assertThat( stripStreamRecordFromWindowedValue(newHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow("finishBundle"), - WindowedValue.valueInGlobalWindow("d"), - WindowedValue.valueInGlobalWindow("finishBundle"))); + WindowedValues.valueInGlobalWindow("finishBundle"), + WindowedValues.valueInGlobalWindow("d"), + WindowedValues.valueInGlobalWindow("finishBundle"))); // close() will also call dispose(), but call again to verify no new bundle // is created afterwards @@ -1644,9 +1646,9 @@ public void finishBundle(FinishBundleContext context) { assertThat( stripStreamRecordFromWindowedValue(newHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow("finishBundle"), - WindowedValue.valueInGlobalWindow("d"), - WindowedValue.valueInGlobalWindow("finishBundle"))); + WindowedValues.valueInGlobalWindow("finishBundle"), + WindowedValues.valueInGlobalWindow("d"), + WindowedValues.valueInGlobalWindow("finishBundle"))); } @Test @@ -1655,8 +1657,8 @@ public void testBundleKeyed() throws Exception { StringUtf8Coder keyCoder = StringUtf8Coder.of(); KvToFlinkKeyKeySelector keySelector = new KvToFlinkKeyKeySelector<>(keyCoder); KvCoder kvCoder = KvCoder.of(keyCoder, StringUtf8Coder.of()); - WindowedValue.ValueOnlyWindowedValueCoder> windowedValueCoder = - WindowedValue.getValueOnlyCoder(kvCoder); + WindowedValues.ValueOnlyWindowedValueCoder> windowedValueCoder = + WindowedValues.getValueOnlyCoder(kvCoder); TupleTag outputTag = new TupleTag<>("main-output"); FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); @@ -1682,7 +1684,7 @@ public void finishBundle(FinishBundleContext context) { DoFnOperator.MultiOutputOutputManagerFactory outputManagerFactory = new DoFnOperator.MultiOutputOutputManagerFactory<>( outputTag, - WindowedValue.getFullCoder(kvCoder.getValueCoder(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(kvCoder.getValueCoder(), GlobalWindow.Coder.INSTANCE), new SerializablePipelineOptions(options)); DoFnOperator, KV, String> doFnOperator = @@ -1711,19 +1713,19 @@ public void finishBundle(FinishBundleContext context) { testHarness.open(); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("key", "a")))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("key", "a")))); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("key", "b")))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("key", "b")))); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("key", "c")))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("key", "c")))); assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow("a"), - WindowedValue.valueInGlobalWindow("b"), - WindowedValue.valueInGlobalWindow("finishBundle"), - WindowedValue.valueInGlobalWindow("c"))); + WindowedValues.valueInGlobalWindow("a"), + WindowedValues.valueInGlobalWindow("b"), + WindowedValues.valueInGlobalWindow("finishBundle"), + WindowedValues.valueInGlobalWindow("c"))); // Take a snapshot OperatorSubtaskState snapshot = testHarness.snapshot(0, 0); @@ -1735,7 +1737,7 @@ public void finishBundle(FinishBundleContext context) { List>> bufferedElements = pushedBackElementsHandler.getElements().collect(Collectors.toList()); assertThat( - bufferedElements, contains(KV.of(0, WindowedValue.valueInGlobalWindow("finishBundle")))); + bufferedElements, contains(KV.of(0, WindowedValues.valueInGlobalWindow("finishBundle")))); testHarness.close(); @@ -1768,7 +1770,7 @@ public void finishBundle(FinishBundleContext context) { // startBundle will output the buffered elements. testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("key", "d")))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("key", "d")))); // check finishBundle by timeout testHarness.setProcessingTime(10); @@ -1777,9 +1779,9 @@ public void finishBundle(FinishBundleContext context) { stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( // The first finishBundle is restored from the checkpoint - WindowedValue.valueInGlobalWindow("finishBundle"), - WindowedValue.valueInGlobalWindow("d"), - WindowedValue.valueInGlobalWindow("finishBundle"))); + WindowedValues.valueInGlobalWindow("finishBundle"), + WindowedValues.valueInGlobalWindow("d"), + WindowedValues.valueInGlobalWindow("finishBundle"))); testHarness.close(); } @@ -1794,13 +1796,13 @@ public void testCheckpointBufferingWithMultipleBundles() throws Exception { TupleTag outputTag = new TupleTag<>("main-output"); StringUtf8Coder coder = StringUtf8Coder.of(); - WindowedValue.ValueOnlyWindowedValueCoder windowedValueCoder = - WindowedValue.getValueOnlyCoder(coder); + WindowedValues.ValueOnlyWindowedValueCoder windowedValueCoder = + WindowedValues.getValueOnlyCoder(coder); DoFnOperator.MultiOutputOutputManagerFactory outputManagerFactory = new DoFnOperator.MultiOutputOutputManagerFactory<>( outputTag, - WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), new SerializablePipelineOptions(options)); Supplier> doFnOperatorSupplier = @@ -1830,7 +1832,7 @@ public void testCheckpointBufferingWithMultipleBundles() throws Exception { // start a bundle testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow("regular element"))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow("regular element"))); // This callback will be executed in the snapshotState function in the course of // finishing the currently active bundle. Everything emitted in the callback should @@ -1842,11 +1844,11 @@ public void testCheckpointBufferingWithMultipleBundles() throws Exception { // the callback which would otherwise cause an infinitive recursion doFnOperator.setBundleFinishedCallback(null); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow("trigger another bundle"))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow("trigger another bundle"))); doFnOperator.invokeFinishBundle(); testHarness.processElement( new StreamRecord<>( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( "check that the previous element is not flushed"))); } catch (Exception e) { throw new RuntimeException(e); @@ -1858,16 +1860,16 @@ public void testCheckpointBufferingWithMultipleBundles() throws Exception { // Check that we have only the element which was emitted before the snapshot assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), - contains(WindowedValue.valueInGlobalWindow("regular element"))); + contains(WindowedValues.valueInGlobalWindow("regular element"))); // Check that we would flush the buffered elements when continuing to run testHarness.processWatermark(Long.MAX_VALUE); assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow("regular element"), - WindowedValue.valueInGlobalWindow("trigger another bundle"), - WindowedValue.valueInGlobalWindow("check that the previous element is not flushed"))); + WindowedValues.valueInGlobalWindow("regular element"), + WindowedValues.valueInGlobalWindow("trigger another bundle"), + WindowedValues.valueInGlobalWindow("check that the previous element is not flushed"))); testHarness.close(); @@ -1879,14 +1881,14 @@ public void testCheckpointBufferingWithMultipleBundles() throws Exception { testHarness2.open(); testHarness2.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow("after restore"))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow("after restore"))); assertThat( stripStreamRecordFromWindowedValue(testHarness2.getOutput()), contains( - WindowedValue.valueInGlobalWindow("trigger another bundle"), - WindowedValue.valueInGlobalWindow("check that the previous element is not flushed"), - WindowedValue.valueInGlobalWindow("after restore"))); + WindowedValues.valueInGlobalWindow("trigger another bundle"), + WindowedValues.valueInGlobalWindow("check that the previous element is not flushed"), + WindowedValues.valueInGlobalWindow("after restore"))); } @Test @@ -1896,8 +1898,8 @@ public void testExactlyOnceBuffering() throws Exception { options.setCheckpointingInterval(1L); TupleTag outputTag = new TupleTag<>("main-output"); - WindowedValue.ValueOnlyWindowedValueCoder windowedValueCoder = - WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + WindowedValues.ValueOnlyWindowedValueCoder windowedValueCoder = + WindowedValues.getValueOnlyCoder(StringUtf8Coder.of()); numStartBundleCalled = 0; DoFn doFn = @@ -1924,7 +1926,7 @@ public void finishBundle(FinishBundleContext context) { DoFnOperator.MultiOutputOutputManagerFactory outputManagerFactory = new DoFnOperator.MultiOutputOutputManagerFactory<>( outputTag, - WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), new SerializablePipelineOptions(options)); Supplier> doFnOperatorSupplier = @@ -1952,8 +1954,8 @@ public void finishBundle(FinishBundleContext context) { testHarness.open(); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("a"))); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("b"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("a"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("b"))); assertThat(Iterables.size(testHarness.getOutput()), is(0)); assertThat(numStartBundleCalled, is(0)); @@ -1966,9 +1968,9 @@ public void finishBundle(FinishBundleContext context) { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow("a"), - WindowedValue.valueInGlobalWindow("b"), - WindowedValue.valueInGlobalWindow("finishBundle"))); + WindowedValues.valueInGlobalWindow("a"), + WindowedValues.valueInGlobalWindow("b"), + WindowedValues.valueInGlobalWindow("finishBundle"))); doFnOperator = doFnOperatorSupplier.get(); testHarness = new OneInputStreamOperatorTestHarness<>(doFnOperator); @@ -1983,9 +1985,9 @@ public void finishBundle(FinishBundleContext context) { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow("a"), - WindowedValue.valueInGlobalWindow("b"), - WindowedValue.valueInGlobalWindow("finishBundle"))); + WindowedValues.valueInGlobalWindow("a"), + WindowedValues.valueInGlobalWindow("b"), + WindowedValues.valueInGlobalWindow("finishBundle"))); // repeat to see if elements are evicted doFnOperator.notifyCheckpointComplete(1L); @@ -1994,9 +1996,9 @@ public void finishBundle(FinishBundleContext context) { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow("a"), - WindowedValue.valueInGlobalWindow("b"), - WindowedValue.valueInGlobalWindow("finishBundle"))); + WindowedValues.valueInGlobalWindow("a"), + WindowedValues.valueInGlobalWindow("b"), + WindowedValues.valueInGlobalWindow("finishBundle"))); } @Test @@ -2007,8 +2009,8 @@ public void testExactlyOnceBufferingFlushDuringDrain() throws Exception { options.setEnableStableInputDrain(true); TupleTag outputTag = new TupleTag<>("main-output"); - WindowedValue.ValueOnlyWindowedValueCoder windowedValueCoder = - WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + WindowedValues.ValueOnlyWindowedValueCoder windowedValueCoder = + WindowedValues.getValueOnlyCoder(StringUtf8Coder.of()); numStartBundleCalled = 0; DoFn doFn = @@ -2035,7 +2037,7 @@ public void finishBundle(FinishBundleContext context) { DoFnOperator.MultiOutputOutputManagerFactory outputManagerFactory = new DoFnOperator.MultiOutputOutputManagerFactory<>( outputTag, - WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), new SerializablePipelineOptions(options)); Supplier> doFnOperatorSupplier = @@ -2063,8 +2065,8 @@ public void finishBundle(FinishBundleContext context) { testHarness.open(); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("a"))); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("b"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("a"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("b"))); assertThat(Iterables.size(testHarness.getOutput()), is(0)); assertThat(numStartBundleCalled, is(0)); @@ -2077,9 +2079,9 @@ public void finishBundle(FinishBundleContext context) { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow("a"), - WindowedValue.valueInGlobalWindow("b"), - WindowedValue.valueInGlobalWindow("finishBundle"))); + WindowedValues.valueInGlobalWindow("a"), + WindowedValues.valueInGlobalWindow("b"), + WindowedValues.valueInGlobalWindow("finishBundle"))); doFnOperator = doFnOperatorSupplier.get(); testHarness = new OneInputStreamOperatorTestHarness<>(doFnOperator); @@ -2102,8 +2104,8 @@ public void testExactlyOnceBufferingKeyed() throws Exception { StringUtf8Coder keyCoder = StringUtf8Coder.of(); KvToFlinkKeyKeySelector keySelector = new KvToFlinkKeyKeySelector<>(keyCoder); KvCoder kvCoder = KvCoder.of(keyCoder, StringUtf8Coder.of()); - WindowedValue.ValueOnlyWindowedValueCoder> windowedValueCoder = - WindowedValue.getValueOnlyCoder(kvCoder); + WindowedValues.ValueOnlyWindowedValueCoder> windowedValueCoder = + WindowedValues.getValueOnlyCoder(kvCoder); DoFn, KV> doFn = new DoFn, KV>() { @@ -2131,7 +2133,7 @@ public void finishBundle(FinishBundleContext context) { DoFnOperator.MultiOutputOutputManagerFactory> outputManagerFactory = new DoFnOperator.MultiOutputOutputManagerFactory<>( outputTag, - WindowedValue.getFullCoder(kvCoder, GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(kvCoder, GlobalWindow.Coder.INSTANCE), new SerializablePipelineOptions(options)); Supplier, KV, KV>> @@ -2165,13 +2167,13 @@ public void finishBundle(FinishBundleContext context) { testHarness.open(); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("key", "a")))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("key", "a")))); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("key", "b")))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("key", "b")))); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("key2", "c")))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("key2", "c")))); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("key2", "d")))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("key2", "d")))); assertThat( testHarness.getOutput() + " should be empty", @@ -2185,11 +2187,11 @@ public void finishBundle(FinishBundleContext context) { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), containsInAnyOrder( - WindowedValue.valueInGlobalWindow(KV.of("key", "a")), - WindowedValue.valueInGlobalWindow(KV.of("key", "b")), - WindowedValue.valueInGlobalWindow(KV.of("key2", "c")), - WindowedValue.valueInGlobalWindow(KV.of("key2", "d")), - WindowedValue.valueInGlobalWindow(KV.of("key3", "finishBundle")))); + WindowedValues.valueInGlobalWindow(KV.of("key", "a")), + WindowedValues.valueInGlobalWindow(KV.of("key", "b")), + WindowedValues.valueInGlobalWindow(KV.of("key2", "c")), + WindowedValues.valueInGlobalWindow(KV.of("key2", "d")), + WindowedValues.valueInGlobalWindow(KV.of("key3", "finishBundle")))); doFnOperator = doFnOperatorSupplier.get(); testHarness = @@ -2206,11 +2208,11 @@ public void finishBundle(FinishBundleContext context) { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), containsInAnyOrder( - WindowedValue.valueInGlobalWindow(KV.of("key", "a")), - WindowedValue.valueInGlobalWindow(KV.of("key", "b")), - WindowedValue.valueInGlobalWindow(KV.of("key2", "c")), - WindowedValue.valueInGlobalWindow(KV.of("key2", "d")), - WindowedValue.valueInGlobalWindow(KV.of("key3", "finishBundle")))); + WindowedValues.valueInGlobalWindow(KV.of("key", "a")), + WindowedValues.valueInGlobalWindow(KV.of("key", "b")), + WindowedValues.valueInGlobalWindow(KV.of("key2", "c")), + WindowedValues.valueInGlobalWindow(KV.of("key2", "d")), + WindowedValues.valueInGlobalWindow(KV.of("key3", "finishBundle")))); // repeat to see if elements are evicted doFnOperator.notifyCheckpointComplete(1L); @@ -2219,11 +2221,11 @@ public void finishBundle(FinishBundleContext context) { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), containsInAnyOrder( - WindowedValue.valueInGlobalWindow(KV.of("key", "a")), - WindowedValue.valueInGlobalWindow(KV.of("key", "b")), - WindowedValue.valueInGlobalWindow(KV.of("key2", "c")), - WindowedValue.valueInGlobalWindow(KV.of("key2", "d")), - WindowedValue.valueInGlobalWindow(KV.of("key3", "finishBundle")))); + WindowedValues.valueInGlobalWindow(KV.of("key", "a")), + WindowedValues.valueInGlobalWindow(KV.of("key", "b")), + WindowedValues.valueInGlobalWindow(KV.of("key2", "c")), + WindowedValues.valueInGlobalWindow(KV.of("key2", "d")), + WindowedValues.valueInGlobalWindow(KV.of("key3", "finishBundle")))); } @Test(expected = IllegalStateException.class) @@ -2233,8 +2235,8 @@ public void testFailOnRequiresStableInputAndDisabledCheckpointing() { StringUtf8Coder keyCoder = StringUtf8Coder.of(); KvToFlinkKeyKeySelector keySelector = new KvToFlinkKeyKeySelector<>(keyCoder); KvCoder kvCoder = KvCoder.of(keyCoder, StringUtf8Coder.of()); - WindowedValue.ValueOnlyWindowedValueCoder> windowedValueCoder = - WindowedValue.getValueOnlyCoder(kvCoder); + WindowedValues.ValueOnlyWindowedValueCoder> windowedValueCoder = + WindowedValues.getValueOnlyCoder(kvCoder); DoFn, KV> doFn = new DoFn, KV>() { @@ -2250,7 +2252,7 @@ public void processElement(ProcessContext context) { DoFnOperator.MultiOutputOutputManagerFactory> outputManagerFactory = new DoFnOperator.MultiOutputOutputManagerFactory<>( outputTag, - WindowedValue.getFullCoder(kvCoder, GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(kvCoder, GlobalWindow.Coder.INSTANCE), new SerializablePipelineOptions(options)); // should make the DoFnOperator creation fail @@ -2282,13 +2284,13 @@ public void testBundleProcessingExceptionIsFatalDuringCheckpointing() throws Exc TupleTag outputTag = new TupleTag<>("main-output"); StringUtf8Coder coder = StringUtf8Coder.of(); - WindowedValue.ValueOnlyWindowedValueCoder windowedValueCoder = - WindowedValue.getValueOnlyCoder(coder); + WindowedValues.ValueOnlyWindowedValueCoder windowedValueCoder = + WindowedValues.getValueOnlyCoder(coder); DoFnOperator.MultiOutputOutputManagerFactory outputManagerFactory = new DoFnOperator.MultiOutputOutputManagerFactory<>( outputTag, - WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), new SerializablePipelineOptions(options)); DoFnOperator doFnOperator = @@ -2321,7 +2323,7 @@ public void finishBundle() { // start a bundle testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow("regular element"))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow("regular element"))); // Make sure we throw Error, not a regular Exception. // A regular exception would just cause the checkpoint to fail. @@ -2371,8 +2373,8 @@ private static DoFnOperator getOperatorForCleanupInspect options.setParallelism(4); TupleTag outputTag = new TupleTag<>("main-output"); - WindowedValue.ValueOnlyWindowedValueCoder windowedValueCoder = - WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + WindowedValues.ValueOnlyWindowedValueCoder windowedValueCoder = + WindowedValues.getValueOnlyCoder(StringUtf8Coder.of()); IdentityDoFn doFn = new IdentityDoFn() { @FinishBundle @@ -2385,7 +2387,7 @@ public void finishBundle(FinishBundleContext context) { DoFnOperator.MultiOutputOutputManagerFactory outputManagerFactory = new DoFnOperator.MultiOutputOutputManagerFactory<>( outputTag, - WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), new SerializablePipelineOptions(options)); return new DoFnOperator<>( @@ -2455,11 +2457,11 @@ public void processElement(ProcessContext c) { private WindowedValue> valuesInWindow( Iterable values, Instant timestamp, BoundedWindow window) { - return WindowedValue.of(values, timestamp, window, PaneInfo.NO_FIRING); + return WindowedValues.of(values, timestamp, window, PaneInfo.NO_FIRING); } private WindowedValue valueInWindow(T value, Instant timestamp, BoundedWindow window) { - return WindowedValue.of(value, timestamp, window, PaneInfo.NO_FIRING); + return WindowedValues.of(value, timestamp, window, PaneInfo.NO_FIRING); } private interface TestHarnessFactory { diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java index f9a2f094f602..982871e59f14 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java @@ -97,10 +97,11 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.NoopLock; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.Struct; @@ -249,7 +250,7 @@ public void sdkErrorsSurfaceOnClose() throws Exception { doThrow(expected).when(bundle).close(); thrown.expectCause(is(expected)); - operator.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow(0))); + operator.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow(0))); testHarness.close(); } @@ -272,9 +273,9 @@ public void expectedInputsAreSent() throws Exception { FnDataReceiver> receiver = Mockito.mock(FnDataReceiver.class); when(bundle.getInputReceivers()).thenReturn(ImmutableMap.of("input", receiver)); - WindowedValue one = WindowedValue.valueInGlobalWindow(1); - WindowedValue two = WindowedValue.valueInGlobalWindow(2); - WindowedValue three = WindowedValue.valueInGlobalWindow(3); + WindowedValue one = WindowedValues.valueInGlobalWindow(1); + WindowedValue two = WindowedValues.valueInGlobalWindow(2); + WindowedValue three = WindowedValues.valueInGlobalWindow(3); OneInputStreamOperatorTestHarness, WindowedValue> testHarness = new OneInputStreamOperatorTestHarness<>(operator); @@ -296,8 +297,8 @@ public void expectedInputsAreSent() throws Exception { @Test public void outputsAreTaggedCorrectly() throws Exception { - WindowedValue.ValueOnlyWindowedValueCoder coder = - WindowedValue.getValueOnlyCoder(VarIntCoder.of()); + WindowedValues.ValueOnlyWindowedValueCoder coder = + WindowedValues.getValueOnlyCoder(VarIntCoder.of()); TupleTag mainOutput = new TupleTag<>("main-output"); TupleTag additionalOutput1 = new TupleTag<>("output-1"); @@ -332,10 +333,10 @@ public void outputsAreTaggedCorrectly() throws Exception { tagsToIds, new SerializablePipelineOptions(FlinkPipelineOptions.defaults())); - WindowedValue zero = WindowedValue.valueInGlobalWindow(0); - WindowedValue three = WindowedValue.valueInGlobalWindow(3); - WindowedValue four = WindowedValue.valueInGlobalWindow(4); - WindowedValue five = WindowedValue.valueInGlobalWindow(5); + WindowedValue zero = WindowedValues.valueInGlobalWindow(0); + WindowedValue three = WindowedValues.valueInGlobalWindow(3); + WindowedValue four = WindowedValues.valueInGlobalWindow(4); + WindowedValue five = WindowedValues.valueInGlobalWindow(5); // We use a real StageBundleFactory here in order to exercise the output receiver factory. StageBundleFactory stageBundleFactory = @@ -462,7 +463,7 @@ public void testWatermarkHandling() throws Exception { outputManagerFactory, WindowingStrategy.of(FixedWindows.of(Duration.millis(10))), StringUtf8Coder.of(), - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()), IntervalWindow.getCoder())); KeyedOneInputStreamOperatorTestHarness< @@ -498,7 +499,7 @@ public void testWatermarkHandling() throws Exception { // Trigger a new bundle IntervalWindow intervalWindow = new IntervalWindow(new Instant(0), new Instant(9)); WindowedValue> windowedValue = - WindowedValue.of(KV.of("one", 1), Instant.now(), intervalWindow, PaneInfo.NO_FIRING); + WindowedValues.of(KV.of("one", 1), Instant.now(), intervalWindow, PaneInfo.NO_FIRING); testHarness.processElement(new StreamRecord<>(windowedValue)); // The output watermark should be held back during the bundle @@ -636,7 +637,7 @@ public void testEnsureStateCleanupWithKeyedInput() throws Exception { outputManagerFactory, WindowingStrategy.globalDefault(), keyCoder, - WindowedValue.getFullCoder(keyCoder, GlobalWindow.Coder.INSTANCE)); + WindowedValues.getFullCoder(keyCoder, GlobalWindow.Coder.INSTANCE)); KeyedOneInputStreamOperatorTestHarness, WindowedValue> testHarness = @@ -760,7 +761,7 @@ private void testEnsureDeferredStateCleanupTimerFiring(boolean withCheckpointing outputManagerFactory, windowingStrategy, keyCoder, - WindowedValue.getFullCoder(kvCoder, windowingStrategy.getWindowFn().windowCoder())); + WindowedValues.getFullCoder(kvCoder, windowingStrategy.getWindowFn().windowCoder())); @SuppressWarnings("unchecked") RemoteBundle bundle = Mockito.mock(RemoteBundle.class); @@ -771,7 +772,7 @@ private void testEnsureDeferredStateCleanupTimerFiring(boolean withCheckpointing IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(1000)); IntervalWindow.IntervalWindowCoder windowCoder = IntervalWindow.IntervalWindowCoder.of(); WindowedValue> windowedValue = - WindowedValue.of( + WindowedValues.of( KV.of("one", 1), window.maxTimestamp(), ImmutableList.of(window), PaneInfo.NO_FIRING); FnDataReceiver receiver = Mockito.mock(FnDataReceiver.class); @@ -933,7 +934,7 @@ public void testEnsureStateCleanupOnFinalWatermark() throws Exception { outputManagerFactory, windowingStrategy, keyCoder, - WindowedValue.getFullCoder(kvCoder, windowCoder)); + WindowedValues.getFullCoder(kvCoder, windowCoder)); KeyedOneInputStreamOperatorTestHarness< FlinkKey, WindowedValue>, WindowedValue> @@ -1094,8 +1095,8 @@ private static ProcessBundleDescriptors.BagUserStateSpec mockBagUserState(String @Test public void testSerialization() { - WindowedValue.ValueOnlyWindowedValueCoder coder = - WindowedValue.getValueOnlyCoder(VarIntCoder.of()); + WindowedValues.ValueOnlyWindowedValueCoder coder = + WindowedValues.getValueOnlyCoder(VarIntCoder.of()); TupleTag mainOutput = new TupleTag<>("main-output"); TupleTag additionalOutput = new TupleTag<>("additional-output"); @@ -1129,7 +1130,7 @@ public void testSerialization() { ExecutableStageDoFnOperator operator = new ExecutableStageDoFnOperator<>( "transform", - WindowedValue.getValueOnlyCoder(VarIntCoder.of()), + WindowedValues.getValueOnlyCoder(VarIntCoder.of()), Collections.emptyMap(), mainOutput, ImmutableList.of(additionalOutput), @@ -1166,7 +1167,7 @@ public void testStableInputApplied() { outputManagerFactory, WindowingStrategy.globalDefault(), null, - WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), stagePayloadWithStableInput, options); @@ -1189,7 +1190,7 @@ private ExecutableStageDoFnOperator getOperator( outputManagerFactory, WindowingStrategy.globalDefault(), null, - WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE)); + WindowedValues.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE)); } private ExecutableStageDoFnOperator getOperator( diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/StreamRecordStripper.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/StreamRecordStripper.java index 004dc83164c2..264761f260ee 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/StreamRecordStripper.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/StreamRecordStripper.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Function; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.FluentIterable; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperatorTest.java index 6380108ddb94..3bee828f23dd 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperatorTest.java @@ -46,10 +46,11 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.AppliedCombineFn; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.flink.api.java.functions.KeySelector; @@ -107,12 +108,12 @@ public void testRestore() throws Exception { assertThat( output, containsInAnyOrder( - WindowedValue.of( + WindowedValues.of( KV.of(1L, 120L), new Instant(9_999), window, PaneInfo.createPane(true, true, ON_TIME)), - WindowedValue.of( + WindowedValues.of( KV.of(2L, 77L), new Instant(9_999), window, @@ -184,9 +185,9 @@ public void testTimerCleanupOfPendingTimerList() throws Exception { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), containsInAnyOrder( - WindowedValue.of( + WindowedValues.of( KV.of(1L, 100L), new Instant(99), window, PaneInfo.createPane(true, true, ON_TIME)), - WindowedValue.of( + WindowedValues.of( KV.of(2L, 150L), new Instant(199), window2, @@ -214,9 +215,9 @@ private WindowDoFnOperator getWindowDoFnOperator(boolean strea SingletonKeyedWorkItemCoder workItemCoder = SingletonKeyedWorkItemCoder.of(VarLongCoder.of(), VarLongCoder.of(), windowCoder); FullWindowedValueCoder> inputCoder = - WindowedValue.getFullCoder(workItemCoder, windowCoder); + WindowedValues.getFullCoder(workItemCoder, windowCoder); FullWindowedValueCoder> outputCoder = - WindowedValue.getFullCoder(KvCoder.of(VarLongCoder.of(), VarLongCoder.of()), windowCoder); + WindowedValues.getFullCoder(KvCoder.of(VarLongCoder.of(), VarLongCoder.of()), windowCoder); FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); options.setStreaming(streaming); @@ -265,7 +266,7 @@ static ItemBuilder builder() { StreamRecord>> toStreamRecord() { WindowedValue> keyedItem = - WindowedValue.of(KV.of(key, value), new Instant(timestamp), window, NO_FIRING); + WindowedValues.of(KV.of(key, value), new Instant(timestamp), window, NO_FIRING); return new StreamRecord<>(keyedItem); } diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java index 8a588894acd3..f57198e08e3e 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java @@ -49,10 +49,10 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.UnboundedReadFromBoundedSource; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.Configuration; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReaderTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReaderTest.java index bf9ea348230a..88c972a70f65 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReaderTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReaderTest.java @@ -33,8 +33,8 @@ import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSourceSplit; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.SourceTestMetrics.TestMetricGroup; import org.apache.beam.sdk.io.Source; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.eventtime.Watermark; import org.apache.flink.api.connector.source.ReaderOutput; import org.apache.flink.api.connector.source.SourceReader; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReaderTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReaderTest.java index 26dd753df0e1..5bb31a910f82 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReaderTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReaderTest.java @@ -41,9 +41,9 @@ import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.eventtime.Watermark; import org.apache.flink.api.connector.source.ReaderOutput; import org.apache.flink.api.connector.source.SourceOutput; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElementsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElementsTest.java index 5652d23662fd..8e8db7496b80 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElementsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElementsTest.java @@ -27,8 +27,8 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.hamcrest.Matchers; import org.joda.time.Instant; @@ -46,14 +46,14 @@ public void testCoder() throws IOException { StringUtf8Coder elementCoder = StringUtf8Coder.of(); // Generics fail to see here that this is Coder org.apache.beam.sdk.coders.Coder windowCoder = GlobalWindow.Coder.INSTANCE; - WindowedValue.WindowedValueCoder windowedValueCoder = - WindowedValue.FullWindowedValueCoder.of(elementCoder, windowCoder); + WindowedValues.WindowedValueCoder windowedValueCoder = + WindowedValues.FullWindowedValueCoder.of(elementCoder, windowCoder); KV key = KV.of("one", 1); BufferedElements.Coder coder = new BufferedElements.Coder(windowedValueCoder, windowCoder, key); BufferedElement element = new BufferedElements.Element( - WindowedValue.of("test", new Instant(2), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); + WindowedValues.of("test", new Instant(2), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); BufferedElement timerElement = new BufferedElements.Timer( "timerId", diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunnerTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunnerTest.java index 58f6fd8484bf..c399185bc5cd 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunnerTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunnerTest.java @@ -28,7 +28,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.flink.api.common.state.ListState; import org.apache.flink.runtime.state.OperatorStateBackend; @@ -168,7 +168,7 @@ private static BufferingDoFnRunner createBufferingDoFnRunner( doFnRunner, "stable-input", StringUtf8Coder.of(), - WindowedValue.getFullCoder(VarIntCoder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(VarIntCoder.of(), GlobalWindow.Coder.INSTANCE), operatorStateBackend, null, concurrentCheckpoints, diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java index 0478a2c5f0a1..e701ab0dbe67 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java @@ -38,7 +38,6 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformReplacements; import org.apache.beam.sdk.util.construction.ReplacementOutputs; import org.apache.beam.sdk.values.KV; @@ -46,6 +45,8 @@ import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; @@ -239,7 +240,7 @@ public PCollection>>>>> expand .setCoder( KvCoder.of( keyCoder, - KvCoder.of(InstantCoder.of(), WindowedValue.getFullCoder(kvCoder, windowCoder)))) + KvCoder.of(InstantCoder.of(), WindowedValues.getFullCoder(kvCoder, windowCoder)))) // Group by key and sort by timestamp, dropping windows as they are reified .apply("PartitionKeys", new GroupByKeyAndSortValuesOnly<>()) @@ -258,7 +259,7 @@ public void processElement(final ProcessContext c, final BoundedWindow window) { KV.of( c.element().getKey(), KV.of( - c.timestamp(), WindowedValue.of(c.element(), c.timestamp(), window, c.pane())))); + c.timestamp(), WindowedValues.of(c.element(), c.timestamp(), window, c.pane())))); } } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java index 6af6c499cfad..8cb82c2ca42d 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java @@ -64,8 +64,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.SystemDoFnInternal; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; @@ -74,6 +72,9 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Function; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; @@ -327,7 +328,7 @@ public void processElement(ProcessContext c, BoundedWindow untypedWindow) throws coder.hash(ImmutableList.of(c.element().getKey())), KV.of( KV.of(c.element().getKey(), window), - WindowedValue.of( + WindowedValues.of( c.element().getValue(), c.timestamp(), untypedWindow, c.pane())))); } } @@ -1024,7 +1025,7 @@ public void processElement(ProcessContext c) throws Exception { c.output( IsmRecord.of( ImmutableList.of(GlobalWindow.INSTANCE, indexInBundle), - WindowedValue.of(c.element(), c.timestamp(), GlobalWindow.INSTANCE, c.pane()))); + WindowedValues.of(c.element(), c.timestamp(), GlobalWindow.INSTANCE, c.pane()))); indexInBundle += 1; } } @@ -1237,7 +1238,7 @@ public void processElement(ProcessContext c, BoundedWindow untypedWindow) throws c.output( KV.of( ismCoderForHash.hash(ImmutableList.of(window)), - KV.of(window, WindowedValue.of(c.element(), c.timestamp(), window, c.pane())))); + KV.of(window, WindowedValues.of(c.element(), c.timestamp(), window, c.pane())))); } } @@ -1361,7 +1362,7 @@ private static WindowedValue valueInEmptyWindows(T value) { return new ValueInEmptyWindows<>(value); } - private static class ValueInEmptyWindows extends WindowedValue { + private static class ValueInEmptyWindows implements WindowedValue { private final T value; @@ -1394,6 +1395,11 @@ public PaneInfo getPane() { return PaneInfo.NO_FIRING; } + @Override + public Iterable> explodeWindows() { + return Collections.emptyList(); + } + @Override public String toString() { return MoreObjects.toStringHelper(getClass()).add("value", getValue()).toString(); diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java index e0e752d0a2e9..08d84705c5c7 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java @@ -93,7 +93,6 @@ import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.DoFnInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.util.construction.SdkComponents; @@ -108,6 +107,7 @@ import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; @@ -675,7 +675,7 @@ public void addOutput(String name, PCollection value) { translator.producers.put(value, translator.currentTransform); // Wrap the PCollection element Coder inside a WindowedValueCoder. Coder coder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( value.getCoder(), value.getWindowingStrategy().getWindowFn().windowCoder()); addOutput(name, value, coder); } @@ -688,7 +688,7 @@ public void addCollectionToSingletonOutput( // The inputValueCoder for the input PCollection should be some // WindowedValueCoder of the input PCollection's element // coder. - checkState(inputValueCoder instanceof WindowedValue.WindowedValueCoder); + checkState(inputValueCoder instanceof WindowedValues.WindowedValueCoder); // The outputValueCoder for the output should be an // IterableCoder of the inputValueCoder. This is a property // of the backend "CollectionToSingleton" step. diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 66cabecb695b..89fd53184101 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -20,7 +20,6 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.beam.sdk.io.gcp.pubsub.PubsubIO.ENABLE_CUSTOM_PUBSUB_SINK; import static org.apache.beam.sdk.io.gcp.pubsub.PubsubIO.ENABLE_CUSTOM_PUBSUB_SOURCE; -import static org.apache.beam.sdk.options.ExperimentalOptions.hasExperiment; import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray; import static org.apache.beam.sdk.util.SerializableUtils.serializeToByteArray; import static org.apache.beam.sdk.util.StringUtils.byteArrayToJsonString; @@ -135,7 +134,6 @@ import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.util.NameUtils; import org.apache.beam.sdk.util.ReleaseInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.ReflectHelpers; import org.apache.beam.sdk.util.construction.BeamUrns; import org.apache.beam.sdk.util.construction.DeduplicatedFlattenFactory; @@ -168,6 +166,7 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptors; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.TextFormat; @@ -2187,7 +2186,7 @@ private static void translate( // Using a GlobalWindowCoder as a place holder because GlobalWindowCoder is known coder. stepContext.addEncodingInput( - WindowedValue.getFullCoder(VoidCoder.of(), GlobalWindow.Coder.INSTANCE)); + WindowedValues.getFullCoder(VoidCoder.of(), GlobalWindow.Coder.INSTANCE)); stepContext.addInput(PropertyNames.PARALLEL_INPUT, input); } } @@ -2244,12 +2243,13 @@ public void translate(Impulse transform, TransformTranslator.TranslationContext } else { StepTranslationContext stepContext = context.addStep(transform, "ParallelRead"); stepContext.addInput(PropertyNames.FORMAT, "impulse"); - WindowedValue.FullWindowedValueCoder coder = - WindowedValue.getFullCoder( + WindowedValues.FullWindowedValueCoder coder = + WindowedValues.getFullCoder( context.getOutput(transform).getCoder(), GlobalWindow.Coder.INSTANCE); byte[] encodedImpulse; try { - encodedImpulse = encodeToByteArray(coder, WindowedValue.valueInGlobalWindow(new byte[0])); + encodedImpulse = + encodeToByteArray(coder, WindowedValues.valueInGlobalWindow(new byte[0])); } catch (Exception e) { throw new RuntimeException(e); } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjectTranslators.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjectTranslators.java index 62351739228f..c0a83c5a8226 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjectTranslators.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjectTranslators.java @@ -43,10 +43,10 @@ import org.apache.beam.sdk.util.InstanceBuilder; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.StringUtils; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** Utilities for creating {@link CloudObjectTranslator} instances for {@link Coder Coders}. */ diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjects.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjects.java index 208b46141f03..f5de65a363f9 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjects.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjects.java @@ -33,9 +33,9 @@ import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.util.construction.Timer; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchViewOverridesTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchViewOverridesTest.java index 72d6786c02d0..1f160ffdf3af 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchViewOverridesTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchViewOverridesTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.dataflow; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -38,10 +38,11 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.joda.time.Instant; @@ -133,26 +134,27 @@ public void testBatchViewAsListToIsmRecordForNonGlobalWindow() throws Exception ImmutableList.of( KV.of( windowA, - WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)), KV.of( windowA, - WindowedValue.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)), KV.of( windowA, - WindowedValue.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)), KV.of( windowB, - WindowedValue.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)), + WindowedValues.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)), KV.of( windowB, - WindowedValue.of(121L, new Instant(14), windowB, PaneInfo.NO_FIRING)))), + WindowedValues.of( + 121L, new Instant(14), windowB, PaneInfo.NO_FIRING)))), KV.of( 2, (Iterable>>) ImmutableList.of( KV.of( windowC, - WindowedValue.of( + WindowedValues.of( 210L, new Instant(25), windowC, PaneInfo.NO_FIRING))))); // The order of the output elements is important relative to processing order @@ -161,22 +163,22 @@ public void testBatchViewAsListToIsmRecordForNonGlobalWindow() throws Exception contains( IsmRecord.of( ImmutableList.of(windowA, 0L), - WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(windowA, 1L), - WindowedValue.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(windowA, 2L), - WindowedValue.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(windowB, 0L), - WindowedValue.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)), + WindowedValues.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(windowB, 1L), - WindowedValue.of(121L, new Instant(14), windowB, PaneInfo.NO_FIRING)), + WindowedValues.of(121L, new Instant(14), windowB, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(windowC, 0L), - WindowedValue.of(210L, new Instant(25), windowC, PaneInfo.NO_FIRING)))); + WindowedValues.of(210L, new Instant(25), windowC, PaneInfo.NO_FIRING)))); } @Test @@ -221,27 +223,27 @@ public void testToIsmRecordForMapLikeDoFn() throws Exception { ImmutableList.of( KV.of( KV.of(1L, windowA), - WindowedValue.of( + WindowedValues.of( 110L, new Instant(1), windowA, PaneInfo.NO_FIRING)), // same window same key as to previous KV.of( KV.of(1L, windowA), - WindowedValue.of( + WindowedValues.of( 111L, new Instant(2), windowA, PaneInfo.NO_FIRING)), // same window different key as to previous KV.of( KV.of(2L, windowA), - WindowedValue.of( + WindowedValues.of( 120L, new Instant(3), windowA, PaneInfo.NO_FIRING)), // different window same key as to previous KV.of( KV.of(2L, windowB), - WindowedValue.of( + WindowedValues.of( 210L, new Instant(11), windowB, PaneInfo.NO_FIRING)), // different window and different key as to previous KV.of( KV.of(3L, windowB), - WindowedValue.of( + WindowedValues.of( 220L, new Instant(12), windowB, PaneInfo.NO_FIRING)))), KV.of( 2, @@ -250,7 +252,7 @@ public void testToIsmRecordForMapLikeDoFn() throws Exception { // different shard KV.of( KV.of(4L, windowC), - WindowedValue.of( + WindowedValues.of( 330L, new Instant(21), windowC, PaneInfo.NO_FIRING))))); // The order of the output elements is important relative to processing order @@ -259,22 +261,22 @@ public void testToIsmRecordForMapLikeDoFn() throws Exception { contains( IsmRecord.of( ImmutableList.of(1L, windowA, 0L), - WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(1L, windowA, 1L), - WindowedValue.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(2L, windowA, 0L), - WindowedValue.of(120L, new Instant(3), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(120L, new Instant(3), windowA, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(2L, windowB, 0L), - WindowedValue.of(210L, new Instant(11), windowB, PaneInfo.NO_FIRING)), + WindowedValues.of(210L, new Instant(11), windowB, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(3L, windowB, 0L), - WindowedValue.of(220L, new Instant(12), windowB, PaneInfo.NO_FIRING)), + WindowedValues.of(220L, new Instant(12), windowB, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(4L, windowC, 0L), - WindowedValue.of(330L, new Instant(21), windowC, PaneInfo.NO_FIRING)))); + WindowedValues.of(330L, new Instant(21), windowC, PaneInfo.NO_FIRING)))); // Verify the number of unique keys per window. assertThat( @@ -351,12 +353,12 @@ public void testToIsmRecordForMapLikeDoFnWithoutUniqueKeysThrowsException() thro ImmutableList.of( KV.of( KV.of(1L, windowA), - WindowedValue.of( + WindowedValues.of( 110L, new Instant(1), windowA, PaneInfo.NO_FIRING)), // same window same key as to previous KV.of( KV.of(1L, windowA), - WindowedValue.of( + WindowedValues.of( 111L, new Instant(2), windowA, PaneInfo.NO_FIRING))))); thrown.expect(IllegalStateException.class); @@ -495,19 +497,19 @@ public void testToMapDoFn() throws Exception { ImmutableList.of( KV.of( windowA, - WindowedValue.of( + WindowedValues.of( KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)), KV.of( windowA, - WindowedValue.of( + WindowedValues.of( KV.of(2L, 21L), new Instant(7), windowA, PaneInfo.NO_FIRING)), KV.of( windowB, - WindowedValue.of( + WindowedValues.of( KV.of(2L, 21L), new Instant(13), windowB, PaneInfo.NO_FIRING)), KV.of( windowB, - WindowedValue.of( + WindowedValues.of( KV.of(3L, 31L), new Instant(15), windowB, @@ -518,7 +520,7 @@ public void testToMapDoFn() throws Exception { ImmutableList.of( KV.of( windowC, - WindowedValue.of( + WindowedValues.of( KV.of(4L, 41L), new Instant(25), windowC, @@ -569,29 +571,29 @@ public void testToMultimapDoFn() throws Exception { ImmutableList.of( KV.of( windowA, - WindowedValue.of( + WindowedValues.of( KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)), // [BEAM-5184] Specifically test with a duplicate value to ensure that // duplicate key/values are not lost. KV.of( windowA, - WindowedValue.of( + WindowedValues.of( KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)), KV.of( windowA, - WindowedValue.of( + WindowedValues.of( KV.of(1L, 12L), new Instant(5), windowA, PaneInfo.NO_FIRING)), KV.of( windowA, - WindowedValue.of( + WindowedValues.of( KV.of(2L, 21L), new Instant(7), windowA, PaneInfo.NO_FIRING)), KV.of( windowB, - WindowedValue.of( + WindowedValues.of( KV.of(2L, 21L), new Instant(13), windowB, PaneInfo.NO_FIRING)), KV.of( windowB, - WindowedValue.of( + WindowedValues.of( KV.of(3L, 31L), new Instant(15), windowB, @@ -602,7 +604,7 @@ public void testToMultimapDoFn() throws Exception { ImmutableList.of( KV.of( windowC, - WindowedValue.of( + WindowedValues.of( KV.of(4L, 41L), new Instant(25), windowC, diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java index 68c0260a9bdd..66e957fbd746 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java @@ -64,12 +64,12 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.util.InstanceBuilder; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.checkerframework.checker.nullness.qual.Nullable; @@ -154,7 +154,7 @@ public static Iterable> data() { .add(IterableCoder.of(VarLongCoder.of())) .add(KvCoder.of(VarLongCoder.of(), ByteArrayCoder.of())) .add( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(VarLongCoder.of(), ByteArrayCoder.of()), IntervalWindow.getCoder())) .add(ByteArrayCoder.of()) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java index 7697271792a8..4a752047e256 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java @@ -31,8 +31,9 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.checkerframework.checker.nullness.qual.Nullable; @@ -111,7 +112,7 @@ public BoundedWindow window() { }); WindowedValue res = - WindowedValue.of(elem.getValue(), elem.getTimestamp(), windows, elem.getPane()); + WindowedValues.of(elem.getValue(), elem.getTimestamp(), windows, elem.getPane()); receiver.process(res); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java index 3667893393fd..8e2b325b580a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java @@ -19,8 +19,8 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoFn; import org.apache.beam.runners.dataflow.worker.util.common.worker.Receiver; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.joda.time.Instant; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java index e917a150e0a2..bd991560c186 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java @@ -32,9 +32,9 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOutputCounter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOutputCounter.java index e4292f2252a2..7c5859a9d324 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOutputCounter.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOutputCounter.java @@ -24,7 +24,7 @@ import org.apache.beam.runners.dataflow.worker.counters.NameContext; import org.apache.beam.runners.dataflow.worker.util.common.worker.ElementCounter; import org.apache.beam.runners.dataflow.worker.util.common.worker.OutputObjectAndByteCounter; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionView.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionView.java index 44907f3631d8..ab3464c4ce53 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionView.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionView.java @@ -30,7 +30,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; @@ -38,6 +37,7 @@ import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java index 22718b745105..741716b82848 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java @@ -30,8 +30,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.joda.time.Instant; @@ -67,7 +68,7 @@ private static WindowedValue> placeIntoElementWindo WindowedValue> compressedElem) { checkTrivialOuterWindows(compressedElem); WindowedValue> res = - WindowedValue.of( + WindowedValues.of( compressedElem.getValue(), BoundedWindow.TIMESTAMP_MIN_VALUE, getUnderlyingWindow(compressedElem.getValue()), diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java index 95bbcddc4cce..d21ff3023914 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java @@ -28,8 +28,9 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.joda.time.Instant; /** @@ -104,7 +105,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - WindowedValue windowed = WindowedValue.of(output, timestamp, windows, pane); + WindowedValue windowed = WindowedValues.of(output, timestamp, windows, pane); outputManager.output(mainOutputTag, windowed); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java index a7610acba280..3424d3fe2380 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java @@ -50,10 +50,10 @@ import org.apache.beam.sdk.transforms.CombineWithContext.Context; import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java index a611769eb5ee..203883f4a34e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java @@ -35,10 +35,10 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReader.java index 8af6068f2a3a..f991c7a588a0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReader.java @@ -49,13 +49,13 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterable; import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterator; import org.apache.beam.sdk.util.common.Reiterable; import org.apache.beam.sdk.util.common.Reiterator; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderFactory.java index daf1e87f4296..e69f41ade145 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderFactory.java @@ -33,8 +33,8 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.ShuffleReadCounterFactory; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderWithFaultyBytesReadCounter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderWithFaultyBytesReadCounter.java index 1540431bb178..d852d18410f2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderWithFaultyBytesReadCounter.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderWithFaultyBytesReadCounter.java @@ -21,8 +21,8 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.ShuffleReadCounterFactory; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorFactory.java index 499dcaa4e77c..91fb640a1757 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorFactory.java @@ -63,9 +63,9 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.fn.IdGenerator; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.graph.MutableNetwork; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReader.java index b19e77cde871..13defcc84212 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReader.java @@ -25,7 +25,7 @@ import org.apache.beam.runners.dataflow.util.RandomAccessData; import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader; import org.apache.beam.sdk.io.fs.ResourceId; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactory.java index d0bf0c96f9a0..d37f2a0f74e5 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactory.java @@ -36,8 +36,8 @@ import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.WeightedValue; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderImpl.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderImpl.java index 86aab6cd1f97..d9430b401416 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderImpl.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderImpl.java @@ -61,7 +61,7 @@ import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.util.VarInt; import org.apache.beam.sdk.util.WeightedValue; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReader.java index c2e8c334a098..687cbe1d7830 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReader.java @@ -66,7 +66,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PCollectionViews.HasDefaultValue; @@ -82,6 +81,7 @@ import org.apache.beam.sdk.values.PCollectionViews.SingletonViewFn; import org.apache.beam.sdk.values.PCollectionViews.SingletonViewFn2; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Function; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSink.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSink.java index 748e3a1542af..43a3a71c6097 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSink.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSink.java @@ -45,7 +45,7 @@ import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.util.VarInt; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.CountingOutputStream; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSinkFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSinkFactory.java index 99d38246b42f..ec650481a333 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSinkFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSinkFactory.java @@ -33,8 +33,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.math.DoubleMath; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java index 2ca04d9168af..6951e3a95b20 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java @@ -32,9 +32,9 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; /** * A {@link ParDoFnFactory} which returns a {@link ParDoFn} that transforms all {@code diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java index ca709de7effb..a7dca89d31e0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java @@ -44,9 +44,10 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.ByteStreams; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.CountingOutputStream; @@ -113,7 +114,7 @@ static ParDoFn create( GroupingTables.bufferingAndSampling( new WindowingCoderGroupingKeyCreator<>(keyCoder), PairInfo.create(), - new CoderSizeEstimator<>(WindowedValue.getValueOnlyCoder(keyCoder)), + new CoderSizeEstimator<>(WindowedValues.getValueOnlyCoder(keyCoder)), new CoderSizeEstimator<>(inputCoder), 0.001, /*sizeEstimatorSampleRate*/ maxSizeBytes /*maxSizeBytes*/); @@ -128,7 +129,7 @@ static ParDoFn create( new WindowingCoderGroupingKeyCreator<>(keyCoder), PairInfo.create(), valueCombiner, - new CoderSizeEstimator<>(WindowedValue.getValueOnlyCoder(keyCoder)), + new CoderSizeEstimator<>(WindowedValues.getValueOnlyCoder(keyCoder)), new CoderSizeEstimator<>(combineFn.getAccumulatorCoder()), 0.001, /*sizeEstimatorSampleRate*/ maxSizeBytes /*maxSizeBytes*/); @@ -242,7 +243,7 @@ public WindowingCoderGroupingKeyCreator(Coder coder) { public Object createGroupingKey(WindowedValue key) throws Exception { // Ignore timestamp for grouping purposes. // The PGBK output will inherit the timestamp of one of its inputs. - return WindowedValue.of( + return WindowedValues.of( coder.structuralValue(key.getValue()), ignored, key.getWindows(), key.getPane()); } } @@ -304,7 +305,7 @@ public void processElement(Object elem) throws Exception { WindowedValue> input = (WindowedValue>) elem; for (BoundedWindow w : input.getWindows()) { WindowedValue> windowsExpandedInput = - WindowedValue.of(input.getValue(), input.getTimestamp(), w, input.getPane()); + WindowedValues.of(input.getValue(), input.getTimestamp(), w, input.getPane()); groupingTable.put(windowsExpandedInput, receiver); } } @@ -361,7 +362,7 @@ public void processElement(Object elem) throws Exception { WindowedValue> input = (WindowedValue>) elem; for (BoundedWindow w : input.getWindows()) { WindowedValue> windowsExpandedInput = - WindowedValue.of(input.getValue(), input.getTimestamp(), w, input.getPane()); + WindowedValues.of(input.getValue(), input.getTimestamp(), w, input.getPane()); if (!sideInputFetcher.storeIfBlocked(windowsExpandedInput)) { groupingTable.put(windowsExpandedInput, receiver); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReader.java index 81dfd59ff2b4..149915533790 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReader.java @@ -28,9 +28,9 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderFactory.java index 1a8c7e7d49ff..5e672c08dddb 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderFactory.java @@ -28,8 +28,8 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSink.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSink.java index b27d26ed21ac..9098cdc6717e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSink.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSink.java @@ -32,7 +32,7 @@ import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubReader.java index 5d2c491ff72b..024b790e8ca9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubReader.java @@ -34,8 +34,9 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; @@ -136,7 +137,7 @@ protected WindowedValue decodeMessage(Windmill.Message message) throws IOExce } else { value = coder.decode(data, Coder.Context.OUTER); } - return WindowedValue.timestampedValueInGlobalWindow( + return WindowedValues.timestampedValueInGlobalWindow( value, WindmillTimeUtils.windmillToHarnessTimestamp(message.getTimestamp())); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubSink.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubSink.java index b4962422b37e..0616f08ae25f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubSink.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubSink.java @@ -35,8 +35,8 @@ import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java index a5971bb6bdaf..39a3bf24fc24 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java @@ -26,9 +26,10 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoFn; import org.apache.beam.runners.dataflow.worker.util.common.worker.Receiver; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.checkerframework.checker.nullness.qual.Nullable; /** A {@link ParDoFnFactory} to create instances of {@link ReifyTimestampAndWindowsParDoFn}. */ @@ -70,10 +71,10 @@ public void processElement(Object untypedElem) throws Exception { WindowedValue> typedElem = (WindowedValue>) untypedElem; receiver.process( - WindowedValue.of( + WindowedValues.of( KV.of( typedElem.getValue().getKey(), - WindowedValue.of( + WindowedValues.of( typedElem.getValue().getValue(), typedElem.getTimestamp(), typedElem.getWindows(), diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSink.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSink.java index 02e2d48c369a..49ab341eee1c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSink.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSink.java @@ -34,9 +34,9 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.primitives.Ints; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkFactory.java index 53175eaf91e5..a03f43defecc 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkFactory.java @@ -28,7 +28,7 @@ import org.apache.beam.runners.dataflow.worker.util.WorkerPropertyNames; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java index 558848f488a7..16b198704449 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java @@ -53,9 +53,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.DoFnInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SplittableProcessFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SplittableProcessFnFactory.java index 4bd130769150..d0461b4165af 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SplittableProcessFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SplittableProcessFnFactory.java @@ -53,10 +53,10 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.DoFnInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.joda.time.Duration; @@ -161,7 +161,7 @@ public void outputWindowedValue( Collection windows, PaneInfo pane) { outputManager.output( - mainOutputTag, WindowedValue.of(output, timestamp, windows, pane)); + mainOutputTag, WindowedValues.of(output, timestamp, windows, pane)); } @Override @@ -171,7 +171,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - outputManager.output(tag, WindowedValue.of(output, timestamp, windows, pane)); + outputManager.output(tag, WindowedValues.of(output, timestamp, windows, pane)); } }, sideInputReader, diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowReshuffleFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowReshuffleFn.java index be14e7cc3bcf..cf334d451ebc 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowReshuffleFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowReshuffleFn.java @@ -25,8 +25,8 @@ import org.apache.beam.runners.dataflow.worker.util.StreamingGroupAlsoByWindowFn; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.ReshuffleTrigger; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java index 36a93f6282bd..e5d8a18be76f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java @@ -34,7 +34,7 @@ import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterDoFnFactory.java index b9bf203e529e..b4c8af80d8cd 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterDoFnFactory.java @@ -32,8 +32,8 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoFn; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; /** * A {@link ParDoFnFactory} which constructs {@link StreamingPCollectionViewWriterParDoFn}s used to diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java index c04ccc383c73..61730b0c8d88 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java @@ -24,8 +24,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java index e01be8035dec..b5b723adb2b5 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java @@ -23,7 +23,7 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.joda.time.Instant; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java index c1a18cbb5dc4..76913baa6aa7 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java @@ -48,8 +48,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.Parser; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; @@ -91,7 +92,7 @@ public StreamingSideInputFetcher( this.blockedMapAddr = blockedMapAddr(mainWindowCoder); this.elementsAddr = StateTags.makeSystemTagInternal( - StateTags.bag("elem", WindowedValue.getFullCoder(inputCoder, mainWindowCoder))); + StateTags.bag("elem", WindowedValues.getFullCoder(inputCoder, mainWindowCoder))); this.oldTimersAddr = StateTags.makeSystemTagInternal(StateTags.bag("timer", TimerDataCoder.of(mainWindowCoder))); this.timersAddr = diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java index 2627441ce39a..f9e2d6de2461 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java @@ -35,9 +35,9 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedWindmillReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedWindmillReader.java index cdce6b88ba66..a56afd636137 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedWindmillReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedWindmillReader.java @@ -32,9 +32,10 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; @@ -124,10 +125,10 @@ protected WindowedValue decodeMessage(Windmill.Message message) throws IOExce @SuppressWarnings("unchecked") T result = (T) KV.of(decode(kvCoder.getKeyCoder(), key), decode(kvCoder.getValueCoder(), data)); - return WindowedValue.of(result, timestampMillis, windows, pane); + return WindowedValues.of(result, timestampMillis, windows, pane); } else { notifyElementRead(data.available() + metadata.available()); - return WindowedValue.of(decode(valueCoder, data), timestampMillis, windows, pane); + return WindowedValues.of(decode(valueCoder, data), timestampMillis, windows, pane); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java index 3fe48870d246..3ddb3c2003db 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java @@ -27,9 +27,9 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.Receiver; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Values; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; /** * A {@link ParDoFnFactory} which returns a {@link ParDoFn} with similar behavior to {@link diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItem.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItem.java index 40ee075f5fb4..29457678576b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItem.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItem.java @@ -34,8 +34,9 @@ import org.apache.beam.sdk.coders.StructuredCoder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Predicate; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Predicates; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.FluentIterable; @@ -110,7 +111,7 @@ public Iterable> elementsIterable() { InputStream inputStream = message.getData().newInput(); ElemT value = valueCoder.decode(inputStream, Coder.Context.OUTER); - return WindowedValue.of(value, timestamp, windows, pane); + return WindowedValues.of(value, timestamp, windows, pane); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBase.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBase.java index 405440ba1193..5e68641bf661 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBase.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBase.java @@ -21,7 +21,7 @@ import java.util.NoSuchElementException; import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java index abdbeb9a469f..a75374ba64af 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java @@ -36,11 +36,11 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.ValueWithRecordId; import org.apache.beam.sdk.values.ValueWithRecordId.ValueWithRecordIdCoder; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java index e7c1cdfe0e9e..d91a5412b917 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java @@ -32,8 +32,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java index b9aecfda611f..6323e757561e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java @@ -59,8 +59,9 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.BackOff; import org.apache.beam.sdk.util.FluentBackoff; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -647,7 +648,7 @@ public boolean advance() throws IOException { @Override public WindowedValue getCurrent() throws NoSuchElementException { - return WindowedValue.timestampedValueInGlobalWindow( + return WindowedValues.timestampedValueInGlobalWindow( reader.getCurrent(), reader.getCurrentTimestamp()); } @@ -858,7 +859,7 @@ public boolean advance() throws IOException { @Override public WindowedValue> getCurrent() throws NoSuchElementException { WindowedValue result = - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( reader.getCurrent(), reader.getCurrentTimestamp()); return result.withValue( new ValueWithRecordId<>(result.getValue(), reader.getCurrentRecordId())); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowAndCombineFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowAndCombineFn.java index c3f1af8d6637..77f45eed23e3 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowAndCombineFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowAndCombineFn.java @@ -36,9 +36,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowFn.java index 516c2b6428a4..5256142f6ad1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowFn.java @@ -18,8 +18,8 @@ package org.apache.beam.runners.dataflow.worker.util; import org.apache.beam.runners.dataflow.worker.GroupAlsoByWindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; /** {@link GroupAlsoByWindowFn} whose input is a {@link KV}, for use in batch. */ public abstract class BatchGroupAlsoByWindowFn diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowReshuffleFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowReshuffleFn.java index 815ac1f822e7..8a1602a50eea 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowReshuffleFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowReshuffleFn.java @@ -24,8 +24,8 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.ReshuffleTrigger; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowViaIteratorsFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowViaIteratorsFn.java index 5e9e79a40bc3..aabbf08be726 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowViaIteratorsFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowViaIteratorsFn.java @@ -32,12 +32,12 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterable; import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterator; import org.apache.beam.sdk.util.common.Reiterable; import org.apache.beam.sdk.util.common.Reiterator; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ArrayListMultimap; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowViaOutputBufferFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowViaOutputBufferFn.java index 67995cdb0ded..767063661149 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowViaOutputBufferFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowViaOutputBufferFn.java @@ -33,9 +33,9 @@ import org.apache.beam.runners.core.triggers.TriggerStateMachines; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/ValueInEmptyWindows.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/ValueInEmptyWindows.java index 5084af0d187e..ca0e2279fb03 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/ValueInEmptyWindows.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/ValueInEmptyWindows.java @@ -22,7 +22,7 @@ import java.util.Objects; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; @@ -37,7 +37,7 @@ * the Dataflow worker uses this in places to pass a {@link WindowedValue} to a method that requires * one without having to provide the global window, which could alter size estimations, etc. */ -public class ValueInEmptyWindows extends WindowedValue { +public class ValueInEmptyWindows implements WindowedValue { private final T value; public ValueInEmptyWindows(T value) { @@ -49,6 +49,11 @@ public PaneInfo getPane() { return PaneInfo.NO_FIRING; } + @Override + public Iterable> explodeWindows() { + return Collections.emptyList(); + } + @Override public T getValue() { return value; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/ComputationWorkExecutorFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/ComputationWorkExecutorFactory.java index d5e0b3a24e2a..269799903300 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/ComputationWorkExecutorFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/ComputationWorkExecutorFactory.java @@ -55,7 +55,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.fn.IdGenerator; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.graph.MutableNetwork; @@ -231,16 +231,16 @@ ComputationWorkExecutor createComputationWorkExecutor( * stage. This encodes many assumptions about how the streaming execution context works. */ private @Nullable Coder extractKeyCoder(Coder readCoder) { - if (!(readCoder instanceof WindowedValue.WindowedValueCoder)) { + if (!(readCoder instanceof WindowedValues.WindowedValueCoder)) { throw new RuntimeException( String.format( "Expected coder for streaming read to be %s, but received %s", - WindowedValue.WindowedValueCoder.class.getSimpleName(), readCoder)); + WindowedValues.WindowedValueCoder.class.getSimpleName(), readCoder)); } // Note that TimerOrElementCoder is a backwards-compatibility class // that is really a FakeKeyedWorkItemCoder - Coder valueCoder = ((WindowedValue.WindowedValueCoder) readCoder).getValueCoder(); + Coder valueCoder = ((WindowedValues.WindowedValueCoder) readCoder).getValueCoder(); if (valueCoder instanceof KvCoder) { return ((KvCoder) valueCoder).getKeyCoder(); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderFactoryTest.java index 58730408b3a6..0c8a3fe36ae4 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderFactoryTest.java @@ -29,7 +29,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.checkerframework.checker.nullness.qual.Nullable; import org.hamcrest.core.IsInstanceOf; import org.junit.Assert; @@ -74,7 +74,7 @@ NativeReader runTestCreateAvroReader( @Test public void testCreatePlainAvroByteReader() throws Exception { Coder coder = - WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE); + WindowedValues.getFullCoder(BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE); NativeReader reader = runTestCreateAvroReader( pathToAvroFile, null, null, CloudObjects.asCloudObject(coder, /*sdkComponents=*/ null)); @@ -90,7 +90,7 @@ public void testCreatePlainAvroByteReader() throws Exception { @Test public void testCreateRichAvroByteReader() throws Exception { Coder coder = - WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE); + WindowedValues.getFullCoder(BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE); NativeReader reader = runTestCreateAvroReader( pathToAvroFile, 200L, 500L, CloudObjects.asCloudObject(coder, /*sdkComponents=*/ null)); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteSinkFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteSinkFactoryTest.java index 34f6f70e8201..59c467663171 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteSinkFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteSinkFactoryTest.java @@ -27,7 +27,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.hamcrest.core.IsInstanceOf; import org.junit.Assert; import org.junit.Test; @@ -58,7 +58,7 @@ private Sink runTestCreateAvroSink(String filename, Coder coder) throws Ex @Test public void testCreateAvroByteSink() throws Exception { Coder coder = - WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE); + WindowedValues.getFullCoder(BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE); Sink sink = runTestCreateAvroSink(pathToAvroFile, coder); assertThat(sink, new IsInstanceOf(AvroByteSink.class)); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/CombineValuesFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/CombineValuesFnFactoryTest.java index 786f3ae76b83..4ca632acc66d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/CombineValuesFnFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/CombineValuesFnFactoryTest.java @@ -50,11 +50,11 @@ import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -235,17 +235,17 @@ public void testCombineValuesFnAll() throws Exception { combineParDoFn.startBundle(receiver); combineParDoFn.processElement( - WindowedValue.valueInGlobalWindow(KV.of("a", Arrays.asList(5, 6, 7)))); + WindowedValues.valueInGlobalWindow(KV.of("a", Arrays.asList(5, 6, 7)))); combineParDoFn.processElement( - WindowedValue.valueInGlobalWindow(KV.of("b", Arrays.asList(1, 3, 7)))); + WindowedValues.valueInGlobalWindow(KV.of("b", Arrays.asList(1, 3, 7)))); combineParDoFn.processElement( - WindowedValue.valueInGlobalWindow(KV.of("c", Arrays.asList(3, 6, 8, 9)))); + WindowedValues.valueInGlobalWindow(KV.of("c", Arrays.asList(3, 6, 8, 9)))); combineParDoFn.finishBundle(); Object[] expectedReceivedElems = { - WindowedValue.valueInGlobalWindow(KV.of("a", String.format("%.1f", 6.0))), - WindowedValue.valueInGlobalWindow(KV.of("b", String.format("%.1f", 3.7))), - WindowedValue.valueInGlobalWindow(KV.of("c", String.format("%.1f", 6.5))), + WindowedValues.valueInGlobalWindow(KV.of("a", String.format("%.1f", 6.0))), + WindowedValues.valueInGlobalWindow(KV.of("b", String.format("%.1f", 3.7))), + WindowedValues.valueInGlobalWindow(KV.of("c", String.format("%.1f", 6.5))), }; assertArrayEquals(expectedReceivedElems, receiver.receivedElems.toArray()); } @@ -268,17 +268,17 @@ public void testCombineValuesFnAdd() throws Exception { combineParDoFn.startBundle(receiver); combineParDoFn.processElement( - WindowedValue.valueInGlobalWindow(KV.of("a", Arrays.asList(5, 6, 7)))); + WindowedValues.valueInGlobalWindow(KV.of("a", Arrays.asList(5, 6, 7)))); combineParDoFn.processElement( - WindowedValue.valueInGlobalWindow(KV.of("b", Arrays.asList(1, 3, 7)))); + WindowedValues.valueInGlobalWindow(KV.of("b", Arrays.asList(1, 3, 7)))); combineParDoFn.processElement( - WindowedValue.valueInGlobalWindow(KV.of("c", Arrays.asList(3, 6, 8, 9)))); + WindowedValues.valueInGlobalWindow(KV.of("c", Arrays.asList(3, 6, 8, 9)))); combineParDoFn.finishBundle(); Object[] expectedReceivedElems = { - WindowedValue.valueInGlobalWindow(KV.of("a", new CountSum(3, 18))), - WindowedValue.valueInGlobalWindow(KV.of("b", new CountSum(3, 11))), - WindowedValue.valueInGlobalWindow(KV.of("c", new CountSum(4, 26))) + WindowedValues.valueInGlobalWindow(KV.of("a", new CountSum(3, 18))), + WindowedValues.valueInGlobalWindow(KV.of("b", new CountSum(3, 11))), + WindowedValues.valueInGlobalWindow(KV.of("c", new CountSum(4, 26))) }; assertArrayEquals(expectedReceivedElems, receiver.receivedElems.toArray()); } @@ -301,17 +301,17 @@ public void testCombineValuesFnMerge() throws Exception { combineParDoFn.startBundle(receiver); combineParDoFn.processElement( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( KV.of( "a", Arrays.asList(new CountSum(3, 6), new CountSum(2, 9), new CountSum(1, 12))))); combineParDoFn.processElement( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( KV.of("b", Arrays.asList(new CountSum(2, 20), new CountSum(1, 1))))); combineParDoFn.finishBundle(); Object[] expectedReceivedElems = { - WindowedValue.valueInGlobalWindow(KV.of("a", new CountSum(6, 27))), - WindowedValue.valueInGlobalWindow(KV.of("b", new CountSum(3, 21))), + WindowedValues.valueInGlobalWindow(KV.of("a", new CountSum(6, 27))), + WindowedValues.valueInGlobalWindow(KV.of("b", new CountSum(3, 21))), }; assertArrayEquals(expectedReceivedElems, receiver.receivedElems.toArray()); } @@ -334,15 +334,15 @@ public void testCombineValuesFnExtract() throws Exception { combineParDoFn.startBundle(receiver); combineParDoFn.processElement( - WindowedValue.valueInGlobalWindow(KV.of("a", new CountSum(6, 27)))); + WindowedValues.valueInGlobalWindow(KV.of("a", new CountSum(6, 27)))); combineParDoFn.processElement( - WindowedValue.valueInGlobalWindow(KV.of("b", new CountSum(3, 21)))); + WindowedValues.valueInGlobalWindow(KV.of("b", new CountSum(3, 21)))); combineParDoFn.finishBundle(); assertArrayEquals( new Object[] { - WindowedValue.valueInGlobalWindow(KV.of("a", String.format("%.1f", 4.5))), - WindowedValue.valueInGlobalWindow(KV.of("b", String.format("%.1f", 7.0))) + WindowedValues.valueInGlobalWindow(KV.of("a", String.format("%.1f", 4.5))), + WindowedValues.valueInGlobalWindow(KV.of("b", String.format("%.1f", 7.0))) }, receiver.receivedElems.toArray()); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactoryTest.java index e95cfc0a0ff5..a43932d243be 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactoryTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.dataflow.worker; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionViewTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionViewTest.java index 413e03f11ff8..f5e6dca64fdd 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionViewTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionViewTest.java @@ -23,9 +23,9 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.transforms.Materializations; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DefaultParDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DefaultParDoFnFactoryTest.java index 371fce54ac65..06138c02653b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DefaultParDoFnFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DefaultParDoFnFactoryTest.java @@ -37,8 +37,8 @@ import org.apache.beam.sdk.util.DoFnInfo; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.StringUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.hamcrest.Matchers; @@ -121,7 +121,7 @@ public void testCreateSimpleParDoFn() throws Exception { SimpleParDoFn simpleParDoFn = (SimpleParDoFn) parDoFn; parDoFn.startBundle(new OutputReceiver()); // DoFnInfo may not yet be initialized until an element is processed - parDoFn.processElement(WindowedValue.valueInGlobalWindow("foo")); + parDoFn.processElement(WindowedValues.valueInGlobalWindow("foo")); @SuppressWarnings("rawtypes") DoFnInfo doFnInfo = simpleParDoFn.getDoFnInfo(); DoFn innerDoFn = (TestDoFn) doFnInfo.getDoFn(); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderTest.java index fa04028b5dd0..cc006d5b1651 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderTest.java @@ -76,10 +76,11 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.Reiterable; import org.apache.beam.sdk.util.common.Reiterator; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.checkerframework.checker.nullness.qual.Nullable; @@ -171,7 +172,7 @@ public void tearDown() throws IOException { private List writeShuffleEntries( List>>> input, boolean sortValues) throws Exception { Coder>>> sinkElemCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( BigEndianIntegerCoder.of(), KvCoder.of(BigEndianIntegerCoder.of(), BigEndianIntegerCoder.of())), @@ -200,7 +201,7 @@ private List writeShuffleEntries( ++kvCount; actualSizes.add( shuffleSinkWriter.add( - WindowedValue.of( + WindowedValues.of( KV.of(key, value), timestamp, Lists.newArrayList(window), @@ -336,7 +337,7 @@ private void runTestReadFromShuffle( ValuesToRead valuesToRead) throws Exception { Coder>>>> sourceElemCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( BigEndianIntegerCoder.of(), IterableCoder.of( @@ -478,7 +479,7 @@ private void runTestBytesReadCounterForOptions( TestShuffleReadCounterFactory shuffleReadCounterFactory = new TestShuffleReadCounterFactory(); Coder>>>> sourceElemCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( BigEndianIntegerCoder.of(), IterableCoder.of( @@ -637,7 +638,7 @@ public void testReadFromShuffleDataAndFailToSplit() throws Exception { null, null, stop, - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( BigEndianIntegerCoder.of(), IterableCoder.of(BigEndianIntegerCoder.of())), IntervalWindow.getCoder()), @@ -701,7 +702,7 @@ public void testConsumedParallelism() throws Exception { null, null, null, - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( BigEndianIntegerCoder.of(), IterableCoder.of(BigEndianIntegerCoder.of())), IntervalWindow.getCoder()), @@ -773,7 +774,7 @@ public void testReadFromShuffleAndDynamicSplit() throws Exception { null, null, null, - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( BigEndianIntegerCoder.of(), IterableCoder.of(BigEndianIntegerCoder.of())), IntervalWindow.getCoder()), @@ -879,7 +880,7 @@ public void testGetApproximateProgress() throws Exception { null, null, null, - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( BigEndianIntegerCoder.of(), IterableCoder.of(BigEndianIntegerCoder.of())), IntervalWindow.getCoder()), @@ -958,7 +959,7 @@ public void testShuffleReadCounterMultipleExecutingSteps() throws Exception { null, null, stop, - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( BigEndianIntegerCoder.of(), IterableCoder.of(BigEndianIntegerCoder.of())), IntervalWindow.getCoder()), diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorFactoryTest.java index 1d83594bb3f6..e77ae309d359 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorFactoryTest.java @@ -94,9 +94,9 @@ import org.apache.beam.sdk.util.DoFnInfo; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.StringUtils; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; @@ -129,7 +129,7 @@ public class IntrinsicMapTaskExecutorFactoryTest { private static final CloudObject windowedStringCoder = CloudObjects.asCloudObject( - WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()), /*sdkComponents=*/ null); + WindowedValues.getValueOnlyCoder(StringUtf8Coder.of()), /*sdkComponents=*/ null); private IntrinsicMapTaskExecutorFactory mapTaskExecutorFactory; private PipelineOptions options; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactoryTest.java index 09dba0fe5fd5..98e51963b10f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactoryTest.java @@ -37,8 +37,8 @@ import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.After; @@ -90,7 +90,7 @@ public void tearDown() throws IOException { @Test public void testFactory() throws Exception { WindowedValueCoder coder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( IsmRecordCoder.of( 1, 0, ImmutableList.>of(StringUtf8Coder.of()), VarLongCoder.of()), GlobalWindow.Coder.INSTANCE); @@ -114,7 +114,7 @@ public void testFactory() throws Exception { @Test public void testFactoryReturnsCachedInstance() throws Exception { Coder coder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( IsmRecordCoder.of( 1, 0, ImmutableList.>of(StringUtf8Coder.of()), VarLongCoder.of()), GlobalWindow.Coder.INSTANCE); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderTest.java index cabf7388875a..65061983c1bf 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderTest.java @@ -70,7 +70,7 @@ import org.apache.beam.sdk.testing.CoderPropertiesTest.NonDeterministicCoder; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.WeightedValue; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Predicate; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Predicates; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReaderTest.java index 0ce92f9d932c..8420977dc47d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReaderTest.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.dataflow.worker; import static org.apache.beam.runners.dataflow.util.Structs.getString; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables.concat; import static org.hamcrest.MatcherAssert.assertThat; @@ -101,11 +101,12 @@ import org.apache.beam.sdk.transforms.windowing.SlidingWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Function; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableListMultimap; @@ -179,7 +180,7 @@ public void tearDown() throws IOException { @Test public void testSingleton() throws Exception { Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); final WindowedValue element = valueInGlobalWindow(42L); final PCollectionView view = Pipeline.create().apply(Create.empty(VarLongCoder.of())).apply(View.asSingleton()); @@ -216,7 +217,7 @@ public void testSingleton() throws Exception { @Test public void testSingletonInWindow() throws Exception { Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); IsmRecordCoder> ismCoder = IsmRecordCoder.of(1, 0, ImmutableList.>of(INTERVAL_WINDOW_CODER), valueCoder); @@ -287,7 +288,7 @@ public void testSingletonMap() throws Exception { 1, 0, ImmutableList.>of(GLOBAL_WINDOW_CODER), - WindowedValue.getFullCoder(mapCoder, GLOBAL_WINDOW_CODER)); + WindowedValues.getFullCoder(mapCoder, GLOBAL_WINDOW_CODER)); final Source source = initInputFile(fromValues(Arrays.asList(element)), recordCoder); final IsmSideInputReader reader = sideInputReader(view.getTagInternal().getId(), source); @@ -323,14 +324,14 @@ public void testSingletonMapInWindow() throws Exception { ImmutableMap.>>builder() .put( firstWindow, - WindowedValue.of( + WindowedValues.of( ImmutableMap.builder().put("foo", 0L).put("bar", -1L).build(), new Instant(7), firstWindow, PaneInfo.NO_FIRING)) .put( secondWindow, - WindowedValue.of( + WindowedValues.of( ImmutableMap.builder().put("bar", -1L).put("baz", 1L).build(), new Instant(53L), secondWindow, @@ -349,7 +350,7 @@ public void testSingletonMapInWindow() throws Exception { 1, 0, ImmutableList.>of(INTERVAL_WINDOW_CODER), - WindowedValue.getFullCoder(mapCoder, INTERVAL_WINDOW_CODER)); + WindowedValues.getFullCoder(mapCoder, INTERVAL_WINDOW_CODER)); final Source source = initInputFile(fromValues(elements.values()), recordCoder); final IsmSideInputReader reader = sideInputReader(view.getTagInternal().getId(), source); @@ -402,7 +403,7 @@ public void testSingletonMultimapInWindow() throws Exception { ImmutableMap.>>>builder() .put( firstWindow, - WindowedValue.of( + WindowedValues.of( (Map) ImmutableListMultimap.builder() .put("foo", 0L) @@ -415,7 +416,7 @@ public void testSingletonMultimapInWindow() throws Exception { PaneInfo.NO_FIRING)) .put( secondWindow, - WindowedValue.of( + WindowedValues.of( (Map) ImmutableListMultimap.builder() .put("bar", -1L) @@ -441,7 +442,7 @@ public void testSingletonMultimapInWindow() throws Exception { 1, 0, ImmutableList.>of(INTERVAL_WINDOW_CODER), - WindowedValue.getFullCoder(mapCoder, INTERVAL_WINDOW_CODER)); + WindowedValues.getFullCoder(mapCoder, INTERVAL_WINDOW_CODER)); final Source source = initInputFile(fromValues(elements.values()), recordCoder); final IsmSideInputReader reader = sideInputReader(view.getTagInternal().getId(), source); @@ -486,7 +487,7 @@ public void testSingletonMultimapInWindow() throws Exception { @Test public void testIterable() throws Exception { Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); IsmRecordCoder> ismCoder = IsmRecordCoder.of( 1, 0, ImmutableList.of(GLOBAL_WINDOW_CODER, BigEndianLongCoder.of()), valueCoder); @@ -536,7 +537,7 @@ public void testIterable() throws Exception { @Test public void testIterableAtN() throws Exception { Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); IsmRecordCoder> ismCoder = IsmRecordCoder.of( 1, 0, ImmutableList.of(GLOBAL_WINDOW_CODER, BigEndianLongCoder.of()), valueCoder); @@ -588,7 +589,7 @@ public void testIterableAtN() throws Exception { @Test public void testIterableInWindow() throws Exception { Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); IsmRecordCoder> ismCoder = IsmRecordCoder.of( 1, 0, ImmutableList.of(INTERVAL_WINDOW_CODER, BigEndianLongCoder.of()), valueCoder); @@ -661,7 +662,7 @@ public void testIterableInWindow() throws Exception { @Test public void testList() throws Exception { Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); IsmRecordCoder> ismCoder = IsmRecordCoder.of( 1, 0, ImmutableList.of(GLOBAL_WINDOW_CODER, BigEndianLongCoder.of()), valueCoder); @@ -713,7 +714,7 @@ public void testList() throws Exception { @Test public void testListInWindow() throws Exception { Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); IsmRecordCoder> ismCoder = IsmRecordCoder.of( 1, 0, ImmutableList.of(INTERVAL_WINDOW_CODER, BigEndianLongCoder.of()), valueCoder); @@ -792,7 +793,7 @@ public void testMap() throws Exception { // Note that we purposely use byte[]s as keys to force structural equality testing // versus using java equality testing. Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); final ListMultimap> elements = ImmutableListMultimap.>builder() .put(new byte[] {0x00}, valueInGlobalWindow(12L)) @@ -874,7 +875,7 @@ public void testMapInWindow() throws Exception { // Note that we purposely use byte[]s as keys to force structural equality testing // versus using java equality testing. Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); final ListMultimap> firstWindow = ImmutableListMultimap.>builder() .put(new byte[] {0x00}, valueInIntervalWindow(12L, 10)) @@ -1000,7 +1001,7 @@ public void testMultimap() throws Exception { // Note that we purposely use byte[]s as keys to force structural equality testing // versus using java equality testing. Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); final ListMultimap> elements = ImmutableListMultimap.>builder() .put(new byte[] {0x00}, valueInGlobalWindow(12L)) @@ -1083,7 +1084,7 @@ public void testMultimapInWindow() throws Exception { // Note that we purposely use byte[]s as keys to force structural equality testing // versus using java equality testing. Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); final ListMultimap> firstWindow = ImmutableListMultimap.>builder() .put(new byte[] {0x00}, valueInIntervalWindow(12L, 10)) @@ -1211,7 +1212,7 @@ public void testMultimapViewInWindow() throws Exception { // the multimap, we specifically use the same instance of the byte[]. byte[] duplicateKey = new byte[] {0x01}; Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); final ListMultimap> firstWindow = ImmutableListMultimap.>builder() .put(new byte[] {0x00}, valueInIntervalWindow(12L, 10)) @@ -1349,7 +1350,7 @@ public void testIterableSideInputReadCounter() throws Exception { // Test startup: Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); IsmRecordCoder> ismCoder = IsmRecordCoder.of( 1, 0, ImmutableList.of(GLOBAL_WINDOW_CODER, BigEndianLongCoder.of()), valueCoder); @@ -1398,7 +1399,7 @@ public void testIterableSideInputReadCounter() throws Exception { @Test public void testIsmReaderReferenceCaching() throws Exception { Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); final WindowedValue element = valueInGlobalWindow(42L); final PCollectionView view = Pipeline.create().apply(Create.empty(VarLongCoder.of())).apply(View.asSingleton()); @@ -1626,7 +1627,7 @@ private static void verifyMap( } WindowedValue valueInIntervalWindow(long value, long startOfWindow) { - return WindowedValue.of( + return WindowedValues.of( value, new Instant(startOfWindow), intervalWindow(startOfWindow), PaneInfo.NO_FIRING); } @@ -1772,7 +1773,7 @@ private Source newIsmSource(IsmRecordCoder> coder, Strin Source source = new Source(); source.setCodec( CloudObjects.asCloudObject( - WindowedValue.getFullCoder(coder, GLOBAL_WINDOW_CODER), /*sdkComponents=*/ null)); + WindowedValues.getFullCoder(coder, GLOBAL_WINDOW_CODER), /*sdkComponents=*/ null)); source.setSpec(new HashMap()); source.getSpec().put(PropertyNames.OBJECT_TYPE_NAME, "IsmSource"); source.getSpec().put(WorkerPropertyNames.FILENAME, tmpFilePath); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSinkTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSinkTest.java index d395ec8acf6d..78c393111c34 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSinkTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSinkTest.java @@ -28,7 +28,7 @@ import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.testing.CoderPropertiesTest.NonDeterministicCoder; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Rule; import org.junit.Test; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactoryTest.java index be1ed3c270c4..fa8ca0e3dd91 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactoryTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.dataflow.worker; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFnsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFnsTest.java index 19827432e134..19af6effb44d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFnsTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFnsTest.java @@ -66,10 +66,11 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.AppliedCombineFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; @@ -114,7 +115,7 @@ public void testPartialGroupByKey() throws Exception { TestOutputReceiver receiver = new TestOutputReceiver( new ElementByteSizeObservableCoder( - WindowedValue.getValueOnlyCoder( + WindowedValues.getValueOnlyCoder( KvCoder.of(keyCoder, IterableCoder.of(valueCoder)))), counterSet, NameContextsForTests.nameContextForTest()); @@ -124,27 +125,27 @@ public void testPartialGroupByKey() throws Exception { GroupingTables.buffering( new WindowingCoderGroupingKeyCreator(keyCoder), PairInfo.create(), - new CoderSizeEstimator(WindowedValue.getValueOnlyCoder(keyCoder)), + new CoderSizeEstimator(WindowedValues.getValueOnlyCoder(keyCoder)), new CoderSizeEstimator(valueCoder)), receiver); pgbkParDoFn.startBundle(receiver); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("hi", 4))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("there", 5))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("hi", 6))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("joe", 7))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("there", 8))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("hi", 9))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("hi", 4))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("there", 5))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("hi", 6))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("joe", 7))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("there", 8))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("hi", 9))); pgbkParDoFn.finishBundle(); assertThat( receiver.outputElems, IsIterableContainingInAnyOrder.containsInAnyOrder( - WindowedValue.valueInGlobalWindow(KV.of("hi", Arrays.asList(4, 6, 9))), - WindowedValue.valueInGlobalWindow(KV.of("there", Arrays.asList(5, 8))), - WindowedValue.valueInGlobalWindow(KV.of("joe", Arrays.asList(7))))); + WindowedValues.valueInGlobalWindow(KV.of("hi", Arrays.asList(4, 6, 9))), + WindowedValues.valueInGlobalWindow(KV.of("there", Arrays.asList(5, 8))), + WindowedValues.valueInGlobalWindow(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. @@ -167,7 +168,7 @@ public void testPartialGroupByKeyWithCombiner() throws Exception { TestOutputReceiver receiver = new TestOutputReceiver( new ElementByteSizeObservableCoder( - WindowedValue.getValueOnlyCoder(KvCoder.of(keyCoder, valueCoder))), + WindowedValues.getValueOnlyCoder(KvCoder.of(keyCoder, valueCoder))), counterSet, NameContextsForTests.nameContextForTest()); @@ -179,27 +180,27 @@ public void testPartialGroupByKeyWithCombiner() throws Exception { new WindowingCoderGroupingKeyCreator(keyCoder), PairInfo.create(), combineFn, - new CoderSizeEstimator(WindowedValue.getValueOnlyCoder(keyCoder)), + new CoderSizeEstimator(WindowedValues.getValueOnlyCoder(keyCoder)), new CoderSizeEstimator(valueCoder)), receiver); pgbkParDoFn.startBundle(receiver); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("hi", 4))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("there", 5))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("hi", 6))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("joe", 7))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("there", 8))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("hi", 9))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("hi", 4))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("there", 5))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("hi", 6))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("joe", 7))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("there", 8))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("hi", 9))); pgbkParDoFn.finishBundle(); assertThat( receiver.outputElems, IsIterableContainingInAnyOrder.containsInAnyOrder( - WindowedValue.valueInGlobalWindow(KV.of("hi", 19)), - WindowedValue.valueInGlobalWindow(KV.of("there", 13)), - WindowedValue.valueInGlobalWindow(KV.of("joe", 7)))); + WindowedValues.valueInGlobalWindow(KV.of("hi", 19)), + WindowedValues.valueInGlobalWindow(KV.of("there", 13)), + WindowedValues.valueInGlobalWindow(KV.of("joe", 7)))); // Exact counter values depend on size of encoded data. If encoding // changes, then these expected counters should change to match. @@ -222,7 +223,7 @@ public void testPartialGroupByKeyWithCombinerAndSideInputs() throws Exception { TestOutputReceiver receiver = new TestOutputReceiver( new ElementByteSizeObservableCoder( - WindowedValue.getValueOnlyCoder(KvCoder.of(keyCoder, valueCoder))), + WindowedValues.getValueOnlyCoder(KvCoder.of(keyCoder, valueCoder))), counterSet, NameContextsForTests.nameContextForTest()); @@ -234,7 +235,7 @@ public void testPartialGroupByKeyWithCombinerAndSideInputs() throws Exception { new WindowingCoderGroupingKeyCreator(keyCoder), PairInfo.create(), combineFn, - new CoderSizeEstimator(WindowedValue.getValueOnlyCoder(keyCoder)), + new CoderSizeEstimator(WindowedValues.getValueOnlyCoder(keyCoder)), new CoderSizeEstimator(valueCoder)), receiver, mockSideInputFetcher); @@ -246,27 +247,27 @@ public void testPartialGroupByKeyWithCombinerAndSideInputs() throws Exception { when(elemsBag.read()) .thenReturn( ImmutableList.of( - WindowedValue.valueInGlobalWindow(KV.of("hi", 4)), - WindowedValue.valueInGlobalWindow(KV.of("there", 5)))); + WindowedValues.valueInGlobalWindow(KV.of("hi", 4)), + WindowedValues.valueInGlobalWindow(KV.of("there", 5)))); when(mockSideInputFetcher.storeIfBlocked( ArgumentMatchers.>>any())) .thenReturn(false, false, false, true); pgbkParDoFn.startBundle(receiver); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("hi", 6))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("joe", 7))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("there", 8))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("hi", 9))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("hi", 6))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("joe", 7))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("there", 8))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("hi", 9))); pgbkParDoFn.finishBundle(); assertThat( receiver.outputElems, IsIterableContainingInAnyOrder.containsInAnyOrder( - WindowedValue.valueInGlobalWindow(KV.of("hi", 10)), - WindowedValue.valueInGlobalWindow(KV.of("there", 13)), - WindowedValue.valueInGlobalWindow(KV.of("joe", 7)))); + WindowedValues.valueInGlobalWindow(KV.of("hi", 10)), + WindowedValues.valueInGlobalWindow(KV.of("there", 13)), + WindowedValues.valueInGlobalWindow(KV.of("joe", 7)))); // Exact counter values depend on size of encoded data. If encoding // changes, then these expected counters should change to match. @@ -291,7 +292,7 @@ public void testCreateWithCombinerAndBatchSideInputs() throws Exception { TestOutputReceiver receiver = new TestOutputReceiver( - new ElementByteSizeObservableCoder(WindowedValue.getValueOnlyCoder(kvCoder)), + new ElementByteSizeObservableCoder(WindowedValues.getValueOnlyCoder(kvCoder)), counterSet, NameContextsForTests.nameContextForTest()); @@ -328,7 +329,7 @@ public void testCreateWithCombinerAndStreaming() throws Exception { TestOutputReceiver receiver = new TestOutputReceiver( - new ElementByteSizeObservableCoder(WindowedValue.getValueOnlyCoder(kvCoder)), + new ElementByteSizeObservableCoder(WindowedValues.getValueOnlyCoder(kvCoder)), counterSet, NameContextsForTests.nameContextForTest()); @@ -355,7 +356,7 @@ public void testCreateWithCombinerAndStreamingSideInputs() throws Exception { TestOutputReceiver receiver = new TestOutputReceiver( - new ElementByteSizeObservableCoder(WindowedValue.getValueOnlyCoder(kvCoder)), + new ElementByteSizeObservableCoder(WindowedValues.getValueOnlyCoder(kvCoder)), counterSet, NameContextsForTests.nameContextForTest()); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderTest.java index 864346278da6..9fe44e5789eb 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderTest.java @@ -34,8 +34,9 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.joda.time.Duration; import org.joda.time.Instant; @@ -55,21 +56,21 @@ public class PartitioningShuffleReaderTest { private static final List>> KVS = Arrays.asList( - WindowedValue.of(KV.of(1, "in 1a"), timestamp, Lists.newArrayList(window), NO_FIRING), - WindowedValue.of(KV.of(1, "in 1b"), timestamp, Lists.newArrayList(window), NO_FIRING), - WindowedValue.of(KV.of(2, "in 2a"), timestamp, Lists.newArrayList(window), NO_FIRING), - WindowedValue.of(KV.of(2, "in 2b"), timestamp, Lists.newArrayList(window), NO_FIRING), - WindowedValue.of(KV.of(3, "in 3"), timestamp, Lists.newArrayList(window), NO_FIRING), - WindowedValue.of(KV.of(4, "in 4a"), timestamp, Lists.newArrayList(window), NO_FIRING), - WindowedValue.of(KV.of(4, "in 4b"), timestamp, Lists.newArrayList(window), NO_FIRING), - WindowedValue.of(KV.of(4, "in 4c"), timestamp, Lists.newArrayList(window), NO_FIRING), - WindowedValue.of(KV.of(4, "in 4d"), timestamp, Lists.newArrayList(window), NO_FIRING), - WindowedValue.of(KV.of(5, "in 5"), timestamp, Lists.newArrayList(window), NO_FIRING)); + WindowedValues.of(KV.of(1, "in 1a"), timestamp, Lists.newArrayList(window), NO_FIRING), + WindowedValues.of(KV.of(1, "in 1b"), timestamp, Lists.newArrayList(window), NO_FIRING), + WindowedValues.of(KV.of(2, "in 2a"), timestamp, Lists.newArrayList(window), NO_FIRING), + WindowedValues.of(KV.of(2, "in 2b"), timestamp, Lists.newArrayList(window), NO_FIRING), + WindowedValues.of(KV.of(3, "in 3"), timestamp, Lists.newArrayList(window), NO_FIRING), + WindowedValues.of(KV.of(4, "in 4a"), timestamp, Lists.newArrayList(window), NO_FIRING), + WindowedValues.of(KV.of(4, "in 4b"), timestamp, Lists.newArrayList(window), NO_FIRING), + WindowedValues.of(KV.of(4, "in 4c"), timestamp, Lists.newArrayList(window), NO_FIRING), + WindowedValues.of(KV.of(4, "in 4d"), timestamp, Lists.newArrayList(window), NO_FIRING), + WindowedValues.of(KV.of(5, "in 5"), timestamp, Lists.newArrayList(window), NO_FIRING)); private void runTestReadFromShuffle(List>> expected) throws Exception { Coder>> elemCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(BigEndianIntegerCoder.of(), StringUtf8Coder.of()), IntervalWindow.getCoder()); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSinkTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSinkTest.java index e258dbe1d85c..d8822ce4937b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSinkTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSinkTest.java @@ -32,7 +32,8 @@ import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.joda.time.Instant; @@ -73,7 +74,7 @@ public void testWriteDynamicDestinations() throws Exception { SinkRegistry.defaultRegistry() .create( cloudSinkSpec, - WindowedValue.getFullCoder(VoidCoder.of(), IntervalWindow.getCoder()), + WindowedValues.getFullCoder(VoidCoder.of(), IntervalWindow.getCoder()), null, mockContext, null) @@ -119,14 +120,14 @@ public void testWriteDynamicDestinations() throws Exception { .toByteString()) .build()); writer.add( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( new PubsubMessage(payload1, null).withTopic("topic1"), new Instant(baseTimestamp))); writer.add( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( new PubsubMessage(payload2, null).withTopic("topic2"), new Instant(baseTimestamp + 1))); writer.add( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( new PubsubMessage(payload3, null).withTopic("topic3"), new Instant(baseTimestamp + 2))); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubReaderTest.java index 4af9555b3bb0..c28fdac04888 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubReaderTest.java @@ -30,7 +30,8 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.joda.time.Instant; import org.junit.Before; @@ -84,7 +85,7 @@ private void testReadWith(String parseFn) throws Exception { (PubsubReader) factory.create( cloudSourceSpec, - WindowedValue.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()), + WindowedValues.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()), null, mockContext, null); @@ -92,13 +93,13 @@ private void testReadWith(String parseFn) throws Exception { NativeReader.NativeReaderIterator> iter = reader.iterator(); assertTrue(iter.start()); assertEquals( - iter.getCurrent(), WindowedValue.timestampedValueInGlobalWindow("e0", new Instant(0))); + iter.getCurrent(), WindowedValues.timestampedValueInGlobalWindow("e0", new Instant(0))); assertTrue(iter.advance()); assertEquals( - iter.getCurrent(), WindowedValue.timestampedValueInGlobalWindow("e1", new Instant(1))); + iter.getCurrent(), WindowedValues.timestampedValueInGlobalWindow("e1", new Instant(1))); assertTrue(iter.advance()); assertEquals( - iter.getCurrent(), WindowedValue.timestampedValueInGlobalWindow("e2", new Instant(2))); + iter.getCurrent(), WindowedValues.timestampedValueInGlobalWindow("e2", new Instant(2))); assertFalse(iter.advance()); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubSinkTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubSinkTest.java index a03f7bc24a6d..5327cd172410 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubSinkTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubSinkTest.java @@ -35,7 +35,8 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.joda.time.Instant; import org.junit.Before; @@ -77,16 +78,19 @@ private void testWriteWith(String formatFn) throws Exception { (PubsubSink) factory.create( cloudSinkSpec, - WindowedValue.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()), + WindowedValues.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()), null, mockContext, null); Sink.SinkWriter> writer = sink.writer(); - assertEquals(2, writer.add(WindowedValue.timestampedValueInGlobalWindow("e0", new Instant(0)))); - assertEquals(2, writer.add(WindowedValue.timestampedValueInGlobalWindow("e1", new Instant(1)))); - assertEquals(2, writer.add(WindowedValue.timestampedValueInGlobalWindow("e2", new Instant(2)))); + assertEquals( + 2, writer.add(WindowedValues.timestampedValueInGlobalWindow("e0", new Instant(0)))); + assertEquals( + 2, writer.add(WindowedValues.timestampedValueInGlobalWindow("e1", new Instant(1)))); + assertEquals( + 2, writer.add(WindowedValues.timestampedValueInGlobalWindow("e2", new Instant(2)))); writer.close(); assertEquals( @@ -161,7 +165,7 @@ public void testExceptionAfterEncoding() throws Exception { (PubsubSink) factory.create( cloudSinkSpec, - WindowedValue.getFullCoder(new ErrorCoder(), IntervalWindow.getCoder()), + WindowedValues.getFullCoder(new ErrorCoder(), IntervalWindow.getCoder()), null, mockContext, null); @@ -170,10 +174,10 @@ public void testExceptionAfterEncoding() throws Exception { assertThrows( "encode error", CoderException.class, - () -> writer.add(WindowedValue.timestampedValueInGlobalWindow("e0", new Instant(0)))); + () -> writer.add(WindowedValues.timestampedValueInGlobalWindow("e0", new Instant(0)))); assertThrows( "encode error", CoderException.class, - () -> writer.add(WindowedValue.timestampedValueInGlobalWindow("e0", new Instant(0)))); + () -> writer.add(WindowedValues.timestampedValueInGlobalWindow("e0", new Instant(0)))); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderFactoryTest.java index ea3d02a099e1..3af59d67e3dd 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderFactoryTest.java @@ -30,7 +30,8 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.checkerframework.checker.nullness.qual.Nullable; import org.hamcrest.CoreMatchers; import org.hamcrest.core.IsInstanceOf; @@ -115,7 +116,7 @@ public boolean advance() throws IOException { @Override public WindowedValue getCurrent() { - return WindowedValue.valueInGlobalWindow("something"); + return WindowedValues.valueInGlobalWindow("something"); } } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderTestUtils.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderTestUtils.java index 343661ae9a6b..473663901c29 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderTestUtils.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderTestUtils.java @@ -31,7 +31,7 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader; import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader.NativeReaderIterator; import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader.Progress; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.checkerframework.checker.nullness.qual.Nullable; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactoryTest.java index 55b8d62261c7..eb8f91af18b9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactoryTest.java @@ -25,8 +25,9 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.Receiver; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; @@ -61,7 +62,7 @@ private void verifyReifiedIsInTheSameWindows(WindowedValue> elem @Test public void testSingleWindow() throws Exception { verifyReifiedIsInTheSameWindows( - WindowedValue.of( + WindowedValues.of( KV.of(42, "bizzle"), new Instant(73), new IntervalWindow(new Instant(5), new Instant(15)), @@ -71,7 +72,7 @@ public void testSingleWindow() throws Exception { @Test public void testMultiWindowStaysCompressed() throws Exception { verifyReifiedIsInTheSameWindows( - WindowedValue.of( + WindowedValues.of( KV.of(42, "bizzle"), new Instant(73), ImmutableList.of( diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleReaderFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleReaderFactoryTest.java index febf8b4380f9..c1d87fb6f78c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleReaderFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleReaderFactoryTest.java @@ -34,8 +34,8 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.checkerframework.checker.nullness.qual.Nullable; import org.hamcrest.core.IsInstanceOf; import org.junit.Assert; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkFactoryTest.java index 0edbf80b4ee1..03a2b28090dc 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkFactoryTest.java @@ -31,8 +31,8 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.hamcrest.core.IsInstanceOf; import org.junit.Assert; import org.junit.Test; @@ -91,7 +91,7 @@ void runTestCreateUngroupingShuffleSink( void runTestCreatePartitioningShuffleSink( byte[] shuffleWriterConfig, Coder keyCoder, Coder valueCoder) throws Exception { FullWindowedValueCoder coder = - WindowedValue.getFullCoder(KvCoder.of(keyCoder, valueCoder), IntervalWindow.getCoder()); + WindowedValues.getFullCoder(KvCoder.of(keyCoder, valueCoder), IntervalWindow.getCoder()); ShuffleSink shuffleSink = runTestCreateShuffleSinkHelper(shuffleWriterConfig, "partition_keys", coder, coder); Assert.assertEquals(ShuffleSink.ShuffleKind.PARTITION_KEYS, shuffleSink.shuffleKind); @@ -110,7 +110,7 @@ void runTestCreatePartitioningShuffleSink( void runTestCreateGroupingShuffleSink( byte[] shuffleWriterConfig, Coder keyCoder, Coder valueCoder) throws Exception { FullWindowedValueCoder coder = - WindowedValue.getFullCoder(KvCoder.of(keyCoder, valueCoder), IntervalWindow.getCoder()); + WindowedValues.getFullCoder(KvCoder.of(keyCoder, valueCoder), IntervalWindow.getCoder()); ShuffleSink shuffleSink = runTestCreateShuffleSinkHelper(shuffleWriterConfig, "group_keys", coder, coder); Assert.assertEquals(ShuffleSink.ShuffleKind.GROUP_KEYS, shuffleSink.shuffleKind); @@ -128,7 +128,7 @@ void runTestCreateGroupingSortingShuffleSink( byte[] shuffleWriterConfig, Coder keyCoder, Coder sortKeyCoder, Coder sortValueCoder) throws Exception { FullWindowedValueCoder coder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(keyCoder, KvCoder.of(sortKeyCoder, sortValueCoder)), IntervalWindow.getCoder()); ShuffleSink shuffleSink = @@ -149,7 +149,7 @@ void runTestCreateGroupingSortingShuffleSink( @Test public void testCreateUngroupingShuffleSink() throws Exception { FullWindowedValueCoder coder = - WindowedValue.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()); + WindowedValues.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()); runTestCreateUngroupingShuffleSink(new byte[] {(byte) 0xE1}, coder, coder); } @@ -164,7 +164,7 @@ public void testCreateGroupingShuffleSink() throws Exception { runTestCreateGroupingShuffleSink( new byte[] {(byte) 0xE2}, BigEndianIntegerCoder.of(), - WindowedValue.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder())); + WindowedValues.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder())); } @Test diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkTest.java index 6db4562847c1..025eb7c2fbb1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkTest.java @@ -36,8 +36,9 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.joda.time.Duration; @@ -87,7 +88,7 @@ public class ShuffleSinkTest { private void runTestWriteUngroupingShuffleSink(List expected) throws Exception { Coder> windowedValueCoder = - WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), new GlobalWindows().windowCoder()); + WindowedValues.getFullCoder(BigEndianIntegerCoder.of(), new GlobalWindows().windowCoder()); BatchModeExecutionContext executionContext = BatchModeExecutionContext.forTesting(PipelineOptionsFactory.create(), "STAGE"); ShuffleSink shuffleSink = @@ -104,7 +105,7 @@ private void runTestWriteUngroupingShuffleSink(List expected) throws Ex try (Sink.SinkWriter> shuffleSinkWriter = shuffleSink.writer(shuffleWriter, "dataset")) { for (Integer value : expected) { - actualSizes.add(shuffleSinkWriter.add(WindowedValue.valueInGlobalWindow(value))); + actualSizes.add(shuffleSinkWriter.add(WindowedValues.valueInGlobalWindow(value))); } } @@ -132,7 +133,7 @@ void runTestWriteGroupingShuffleSink(List> expected) throws PipelineOptionsFactory.create(), null, ShuffleSink.ShuffleKind.GROUP_KEYS, - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(BigEndianIntegerCoder.of(), StringUtf8Coder.of()), IntervalWindow.getCoder()), executionContext, @@ -145,7 +146,7 @@ void runTestWriteGroupingShuffleSink(List> expected) throws for (KV kv : expected) { actualSizes.add( shuffleSinkWriter.add( - WindowedValue.of( + WindowedValues.of( KV.of(kv.getKey(), kv.getValue()), timestamp, Lists.newArrayList(window), @@ -181,7 +182,7 @@ void runTestWriteGroupingSortingShuffleSink(List PipelineOptionsFactory.create(), null, ShuffleSink.ShuffleKind.GROUP_KEYS_AND_SORT_VALUES, - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( BigEndianIntegerCoder.of(), KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())), @@ -194,7 +195,7 @@ void runTestWriteGroupingSortingShuffleSink(List try (Sink.SinkWriter>>> shuffleSinkWriter = shuffleSink.writer(shuffleWriter, "dataset")) { for (KV> kv : expected) { - actualSizes.add(shuffleSinkWriter.add(WindowedValue.valueInGlobalWindow(kv))); + actualSizes.add(shuffleSinkWriter.add(WindowedValues.valueInGlobalWindow(kv))); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFnTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFnTest.java index 76cc74673fb0..bb92fca3d8be 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFnTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFnTest.java @@ -55,9 +55,9 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.DoFnInfo; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @@ -236,41 +236,41 @@ public void testOutputReceivers() throws Exception { userParDoFn.startBundle(receiver, receiver1, receiver2, receiver3); - userParDoFn.processElement(WindowedValue.valueInGlobalWindow(3)); - userParDoFn.processElement(WindowedValue.valueInGlobalWindow(42)); - userParDoFn.processElement(WindowedValue.valueInGlobalWindow(666)); + userParDoFn.processElement(WindowedValues.valueInGlobalWindow(3)); + userParDoFn.processElement(WindowedValues.valueInGlobalWindow(42)); + userParDoFn.processElement(WindowedValues.valueInGlobalWindow(666)); userParDoFn.finishBundle(); Object[] expectedReceivedElems = { - WindowedValue.valueInGlobalWindow("processing: 3"), - WindowedValue.valueInGlobalWindow("processing: 42"), - WindowedValue.valueInGlobalWindow("processing: 666"), - WindowedValue.valueInGlobalWindow("finished"), + WindowedValues.valueInGlobalWindow("processing: 3"), + WindowedValues.valueInGlobalWindow("processing: 42"), + WindowedValues.valueInGlobalWindow("processing: 666"), + WindowedValues.valueInGlobalWindow("finished"), }; assertArrayEquals(expectedReceivedElems, receiver.receivedElems.toArray()); Object[] expectedReceivedElems1 = { - WindowedValue.valueInGlobalWindow("tag1: processing: 3"), - WindowedValue.valueInGlobalWindow("tag1: processing: 42"), - WindowedValue.valueInGlobalWindow("tag1: processing: 666"), - WindowedValue.valueInGlobalWindow("tag1: finished"), + WindowedValues.valueInGlobalWindow("tag1: processing: 3"), + WindowedValues.valueInGlobalWindow("tag1: processing: 42"), + WindowedValues.valueInGlobalWindow("tag1: processing: 666"), + WindowedValues.valueInGlobalWindow("tag1: finished"), }; assertArrayEquals(expectedReceivedElems1, receiver1.receivedElems.toArray()); Object[] expectedReceivedElems2 = { - WindowedValue.valueInGlobalWindow("tag2: processing: 3"), - WindowedValue.valueInGlobalWindow("tag2: processing: 42"), - WindowedValue.valueInGlobalWindow("tag2: processing: 666"), - WindowedValue.valueInGlobalWindow("tag2: finished"), + WindowedValues.valueInGlobalWindow("tag2: processing: 3"), + WindowedValues.valueInGlobalWindow("tag2: processing: 42"), + WindowedValues.valueInGlobalWindow("tag2: processing: 666"), + WindowedValues.valueInGlobalWindow("tag2: finished"), }; assertArrayEquals(expectedReceivedElems2, receiver2.receivedElems.toArray()); Object[] expectedReceivedElems3 = { - WindowedValue.valueInGlobalWindow("tag3: processing: 3"), - WindowedValue.valueInGlobalWindow("tag3: processing: 42"), - WindowedValue.valueInGlobalWindow("tag3: processing: 666"), - WindowedValue.valueInGlobalWindow("tag3: finished"), + WindowedValues.valueInGlobalWindow("tag3: processing: 3"), + WindowedValues.valueInGlobalWindow("tag3: processing: 42"), + WindowedValues.valueInGlobalWindow("tag3: processing: 666"), + WindowedValues.valueInGlobalWindow("tag3: finished"), }; assertArrayEquals(expectedReceivedElems3, receiver3.receivedElems.toArray()); } @@ -381,7 +381,7 @@ public void testErrorPropagation() throws Exception { } try { - userParDoFn.processElement(WindowedValue.valueInGlobalWindow(3)); + userParDoFn.processElement(WindowedValues.valueInGlobalWindow(3)); fail("should have failed"); } catch (Exception exn) { // Exception should be a UserCodeException since we're calling @@ -453,7 +453,7 @@ public void testUndeclaredSideOutputs() throws Exception { thrown.expect(UserCodeException.class); thrown.expectCause(instanceOf(IllegalArgumentException.class)); thrown.expectMessage("Unknown output tag"); - userParDoFn.processElement(WindowedValue.valueInGlobalWindow(5)); + userParDoFn.processElement(WindowedValues.valueInGlobalWindow(5)); } @Test @@ -513,7 +513,7 @@ public void processElement(ProcessContext c) throws Exception { // This test ensures proper behavior of the state sampling even with lazy initialization. try (Closeable trackerCloser = tracker.activate()) { try (Closeable processCloser = operationContext.enterProcess()) { - userParDoFn.processElement(WindowedValue.valueInGlobalWindow(5)); + userParDoFn.processElement(WindowedValues.valueInGlobalWindow(5)); } } } @@ -603,7 +603,7 @@ public void processElement(ProcessContext c) { parDoFn.startBundle(new TestReceiver()); for (int input : inputData) { - parDoFn.processElement(WindowedValue.valueInGlobalWindow(input)); + parDoFn.processElement(WindowedValues.valueInGlobalWindow(input)); } return operationContext diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java index 0cf1bb330dad..11dd51f7fb3a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java @@ -169,14 +169,15 @@ import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.StringUtils; import org.apache.beam.sdk.util.VarInt; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; @@ -434,7 +435,7 @@ private ParallelInstruction makeSourceInstruction(Coder coder) { .setSpec(CloudObject.forClass(UngroupedWindmillReader.class)) .setCodec( CloudObjects.asCloudObject( - WindowedValue.getFullCoder(coder, IntervalWindow.getCoder()), + WindowedValues.getFullCoder(coder, IntervalWindow.getCoder()), /* sdkComponents= */ null)))) .setOutputs( Collections.singletonList( @@ -444,7 +445,7 @@ private ParallelInstruction makeSourceInstruction(Coder coder) { .setSystemName(DEFAULT_OUTPUT_SYSTEM_NAME) .setCodec( CloudObjects.asCloudObject( - WindowedValue.getFullCoder(coder, IntervalWindow.getCoder()), + WindowedValues.getFullCoder(coder, IntervalWindow.getCoder()), /* sdkComponents= */ null)))); } @@ -489,7 +490,7 @@ private ParallelInstruction makeDoFnInstruction( .setSystemName(DEFAULT_OUTPUT_SYSTEM_NAME) .setCodec( CloudObjects.asCloudObject( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( outputCoder, windowingStrategy.getWindowFn().windowCoder()), /* sdkComponents= */ null)))); } @@ -526,7 +527,7 @@ private ParallelInstruction makeSinkInstruction( .setSpec(spec) .setCodec( CloudObjects.asCloudObject( - WindowedValue.getFullCoder(coder, windowCoder), + WindowedValues.getFullCoder(coder, windowCoder), /* sdkComponents= */ null)))); } @@ -1562,7 +1563,7 @@ public void testAssignWindows() throws Exception { .setName("output") .setCodec( CloudObjects.asCloudObject( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( StringUtf8Coder.of(), IntervalWindow.getCoder()), /* sdkComponents= */ null)))); @@ -2401,7 +2402,7 @@ private List makeUnboundedSourcePipeline( options.setNumWorkers(1); CloudObject codec = CloudObjects.asCloudObject( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( ValueWithRecordId.ValueWithRecordIdCoder.of( KvCoder.of(VarIntCoder.of(), VarIntCoder.of())), GlobalWindow.Coder.INSTANCE), @@ -3241,7 +3242,7 @@ public void testExceptionInvalidatesCache() throws Exception { CloudObject codec = CloudObjects.asCloudObject( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( ValueWithRecordId.ValueWithRecordIdCoder.of( KvCoder.of(VarIntCoder.of(), VarIntCoder.of())), GlobalWindow.Coder.INSTANCE), diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowFnsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowFnsTest.java index 5ae4ab020f20..c27b386a76c8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowFnsTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowFnsTest.java @@ -72,9 +72,9 @@ import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.joda.time.Duration; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java index 2069899aa699..3574aa22b70d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java @@ -44,9 +44,9 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.hamcrest.Matchers; import org.joda.time.Instant; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunnerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunnerTest.java index 9d850a8bd935..c400c686f764 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunnerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunnerTest.java @@ -50,9 +50,10 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.AppliedCombineFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; @@ -156,7 +157,7 @@ public void testStartBundle() throws Exception { private WindowedValue createDatum(T element, long timestampMillis) { Instant timestamp = new Instant(timestampMillis); - return WindowedValue.of( + return WindowedValues.of( element, timestamp, Arrays.asList(WINDOW_FN.assignWindow(timestamp)), PaneInfo.NO_FIRING); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterDoFnFactoryTest.java index 374d4ac6308d..fcf7e4d7deb9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterDoFnFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterDoFnFactoryTest.java @@ -29,7 +29,7 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoFn; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.Test; import org.junit.runner.RunWith; @@ -52,7 +52,7 @@ public void testConstruction() throws Exception { CloudObject coder = CloudObjects.asCloudObject( - WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE), /*sdkComponents=*/ null); ParDoFn parDoFn = new StreamingPCollectionViewWriterDoFnFactory() diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java index 2ea8e0bcd9de..9a6544122c69 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java @@ -59,9 +59,10 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -207,7 +208,7 @@ public void testMultipleWindowsNotReady() throws Exception { long timestamp = 1L; WindowedValue elem = - WindowedValue.of( + WindowedValues.of( "e", new Instant(timestamp), Arrays.asList(window1, window2), PaneInfo.NO_FIRING); runner.startBundle(); @@ -449,7 +450,7 @@ private PCollectionView createView() { } private WindowedValue createDatum(String element, long timestamp) { - return WindowedValue.of( + return WindowedValues.of( element, new Instant(timestamp), Arrays.asList(createWindow(timestamp)), diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java index 0aea96f2f906..0f5cd1a0d233 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java @@ -45,9 +45,10 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -197,7 +198,7 @@ private PCollectionView createView() { } private WindowedValue createDatum(String element, long timestamp) { - return WindowedValue.of( + return WindowedValues.of( element, new Instant(timestamp), Arrays.asList(createWindow(timestamp)), diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactoryTest.java index 02ca8e923a23..006d61493c08 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactoryTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.dataflow.worker; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReaderTest.java index 1e504d97b81d..5e8fe648e7d7 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReaderTest.java @@ -30,7 +30,8 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.joda.time.Duration; import org.joda.time.Instant; @@ -48,7 +49,7 @@ public class UngroupedShuffleReaderTest { void runTestReadFromShuffle(List expected) throws Exception { Coder> elemCoder = - WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), IntervalWindow.getCoder()); + WindowedValues.getFullCoder(BigEndianIntegerCoder.of(), IntervalWindow.getCoder()); BatchModeExecutionContext executionContext = BatchModeExecutionContext.forTesting(PipelineOptionsFactory.create(), "STAGE"); @@ -70,7 +71,7 @@ void runTestReadFromShuffle(List expected) throws Exception { for (Integer value : expected) { actualSizes.add( shuffleSinkWriter.add( - WindowedValue.of( + WindowedValues.of( value, timestamp, Lists.newArrayList(window), PaneInfo.NO_FIRING))); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UserParDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UserParDoFnFactoryTest.java index 17f48e71581d..771ffe192baf 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UserParDoFnFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UserParDoFnFactoryTest.java @@ -65,9 +65,9 @@ import org.apache.beam.sdk.util.DoFnInfo; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.StringUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.joda.time.Duration; @@ -193,7 +193,7 @@ public void testFactoryReuseInStep() throws Exception { Receiver rcvr = new OutputReceiver(); parDoFn.startBundle(rcvr); - parDoFn.processElement(WindowedValue.valueInGlobalWindow("foo")); + parDoFn.processElement(WindowedValues.valueInGlobalWindow("foo")); TestDoFn fn = (TestDoFn) ((SimpleParDoFn) parDoFn).getDoFnInfo().getDoFn(); assertThat(fn, not(theInstance(initialFn))); @@ -215,7 +215,7 @@ public void testFactoryReuseInStep() throws Exception { assertThat(fn.state, equalTo(TestDoFn.State.FINISHED)); secondParDoFn.startBundle(rcvr); - secondParDoFn.processElement(WindowedValue.valueInGlobalWindow("spam")); + secondParDoFn.processElement(WindowedValues.valueInGlobalWindow("spam")); TestDoFn reobtainedFn = (TestDoFn) ((SimpleParDoFn) secondParDoFn).getDoFnInfo().getDoFn(); secondParDoFn.finishBundle(); assertThat(reobtainedFn.state, equalTo(TestDoFn.State.FINISHED)); @@ -252,13 +252,13 @@ public void testFactorySimultaneousUse() throws Exception { Receiver rcvr = new OutputReceiver(); parDoFn.startBundle(rcvr); - parDoFn.processElement(WindowedValue.valueInGlobalWindow("foo")); + parDoFn.processElement(WindowedValues.valueInGlobalWindow("foo")); // Must be after the first call to process element for reallyStartBundle to have been called TestDoFn firstDoFn = (TestDoFn) ((SimpleParDoFn) parDoFn).getDoFnInfo().getDoFn(); secondParDoFn.startBundle(rcvr); - secondParDoFn.processElement(WindowedValue.valueInGlobalWindow("spam")); + secondParDoFn.processElement(WindowedValues.valueInGlobalWindow("spam")); // Must be after the first call to process element for reallyStartBundle to have been called TestDoFn secondDoFn = (TestDoFn) ((SimpleParDoFn) secondParDoFn).getDoFnInfo().getDoFn(); @@ -290,7 +290,7 @@ public void testFactoryDoesNotReuseAfterAborted() throws Exception { Receiver rcvr = new OutputReceiver(); parDoFn.startBundle(rcvr); - parDoFn.processElement(WindowedValue.valueInGlobalWindow("foo")); + parDoFn.processElement(WindowedValues.valueInGlobalWindow("foo")); TestDoFn fn = (TestDoFn) ((SimpleParDoFn) parDoFn).getDoFnInfo().getDoFn(); parDoFn.abort(); @@ -308,7 +308,7 @@ public void testFactoryDoesNotReuseAfterAborted() throws Exception { TestOperationContext.create(counters)); secondParDoFn.startBundle(rcvr); - secondParDoFn.processElement(WindowedValue.valueInGlobalWindow("foo")); + secondParDoFn.processElement(WindowedValues.valueInGlobalWindow("foo")); TestDoFn secondFn = (TestDoFn) ((SimpleParDoFn) secondParDoFn).getDoFnInfo().getDoFn(); assertThat(secondFn, not(theInstance(fn))); @@ -378,7 +378,7 @@ public void testCleanupRegistered() throws Exception { IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(10)); parDoFn.processElement( - WindowedValue.of("foo", new Instant(1), firstWindow, PaneInfo.NO_FIRING)); + WindowedValues.of("foo", new Instant(1), firstWindow, PaneInfo.NO_FIRING)); verify(stepContext) .setStateCleanupTimer( @@ -435,7 +435,7 @@ public void testCleanupTimerForGlobalWindowWithAllowedLateness() throws Exceptio GlobalWindow globalWindow = GlobalWindow.INSTANCE; parDoFn.processElement( - WindowedValue.of("foo", new Instant(1), globalWindow, PaneInfo.NO_FIRING)); + WindowedValues.of("foo", new Instant(1), globalWindow, PaneInfo.NO_FIRING)); assertThat( globalWindow.maxTimestamp().plus(allowedLateness), diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactoryTest.java index c0e5a301ea2a..5e5f4e61a466 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactoryTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.dataflow.worker; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java index 1dc4ba111110..546b33e42f1f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java @@ -39,7 +39,7 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.hamcrest.Matchers; import org.joda.time.Instant; @@ -97,9 +97,9 @@ public void testElementIteration() throws Exception { assertThat( keyedWorkItem.elementsIterable(), Matchers.contains( - WindowedValue.of("hello", new Instant(5), WINDOW_1, paneInfo(0)), - WindowedValue.of("world", new Instant(7), WINDOW_2, paneInfo(2)), - WindowedValue.of("earth", new Instant(6), WINDOW_1, paneInfo(1)))); + WindowedValues.of("hello", new Instant(5), WINDOW_1, paneInfo(0)), + WindowedValues.of("world", new Instant(7), WINDOW_2, paneInfo(2)), + WindowedValues.of("earth", new Instant(6), WINDOW_1, paneInfo(1)))); } private void addElement( diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBaseTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBaseTest.java index ebd51a0fba63..b6a4cb86c686 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBaseTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBaseTest.java @@ -25,7 +25,8 @@ import java.util.Arrays; import java.util.List; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.junit.Test; import org.junit.runner.RunWith; @@ -41,7 +42,7 @@ protected TestWindmillReaderIterator(Windmill.WorkItem work) { @Override protected WindowedValue decodeMessage(Windmill.Message message) { - return WindowedValue.valueInGlobalWindow(message.getTimestamp()); + return WindowedValues.valueInGlobalWindow(message.getTimestamp()); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java index 2f63393218c5..df3b959c82c5 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java @@ -30,7 +30,7 @@ import static org.apache.beam.sdk.testing.SourceTestUtils.readFromSource; import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray; import static org.apache.beam.sdk.util.SerializableUtils.deserializeFromByteArray; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Throwables.getStackTraceAsString; import static org.hamcrest.MatcherAssert.assertThat; @@ -119,13 +119,13 @@ import org.apache.beam.sdk.testing.ExpectedLogs; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/LengthPrefixUnknownCodersTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/LengthPrefixUnknownCodersTest.java index 55cb0636e35d..c306ccaa7b9c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/LengthPrefixUnknownCodersTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/LengthPrefixUnknownCodersTest.java @@ -53,8 +53,9 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.graph.MutableNetwork; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.graph.NetworkBuilder; @@ -68,17 +69,17 @@ @RunWith(JUnit4.class) public class LengthPrefixUnknownCodersTest { private static final Coder>> windowedValueCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()), GlobalWindow.Coder.INSTANCE); private static final Coder>> prefixedWindowedValueCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(StringUtf8Coder.of(), LengthPrefixCoder.of(VarIntCoder.of())), GlobalWindow.Coder.INSTANCE); private static final Coder>> prefixedAndReplacedWindowedValueCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(StringUtf8Coder.of(), LENGTH_PREFIXED_BYTE_ARRAY_CODER), GlobalWindow.Coder.INSTANCE); @@ -106,7 +107,7 @@ public void testLengthPrefixUnknownCoders() throws Exception { @Test public void testLengthPrefixForLengthPrefixCoder() throws Exception { Coder>> windowedValueCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(StringUtf8Coder.of(), LengthPrefixCoder.of(VarIntCoder.of())), GlobalWindow.Coder.INSTANCE); @@ -114,7 +115,7 @@ public void testLengthPrefixForLengthPrefixCoder() throws Exception { forCodec(CloudObjects.asCloudObject(windowedValueCoder, /*sdkComponents=*/ null), false); Coder>> expectedCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(StringUtf8Coder.of(), LengthPrefixCoder.of(VarIntCoder.of())), GlobalWindow.Coder.INSTANCE); @@ -127,7 +128,7 @@ public void testLengthPrefixForLengthPrefixCoder() throws Exception { @Test public void testLengthPrefixAndReplaceUnknownCoder() throws Exception { Coder>> windowedValueCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()), GlobalWindow.Coder.INSTANCE); Map lengthPrefixedCoderCloudObject = diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowReshuffleDoFnTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowReshuffleDoFnTest.java index 3d2052ca84fc..613fa619452b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowReshuffleDoFnTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowReshuffleDoFnTest.java @@ -38,9 +38,10 @@ import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; @@ -158,11 +159,11 @@ public void testReshuffleFiresEveryElement() throws Exception { gabwFactory, windowingStrategy, "key", - WindowedValue.of( + WindowedValues.of( "v1", new Instant(1), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v2", new Instant(2), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v3", new Instant(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING)); assertThat(result.size(), equalTo(3)); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/CombiningGroupAlsoByWindowsViaOutputBufferDoFnTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/CombiningGroupAlsoByWindowsViaOutputBufferDoFnTest.java index eeb9f83b244d..cc69a91828c7 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/CombiningGroupAlsoByWindowsViaOutputBufferDoFnTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/CombiningGroupAlsoByWindowsViaOutputBufferDoFnTest.java @@ -41,7 +41,7 @@ @RunWith(JUnit4.class) public class CombiningGroupAlsoByWindowsViaOutputBufferDoFnTest { - private class CombiningGABWViaOutputBufferDoFnFactory + private static class CombiningGABWViaOutputBufferDoFnFactory implements GroupAlsoByWindowDoFnFactory { private final Coder keyCoder; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/GroupAlsoByWindowProperties.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/GroupAlsoByWindowProperties.java index 8a42107e09f4..2ade098c9ef8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/GroupAlsoByWindowProperties.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/GroupAlsoByWindowProperties.java @@ -43,10 +43,11 @@ import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.transforms.windowing.SlidingWindows; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheLoader; @@ -116,11 +117,11 @@ public static void groupsElementsIntoFixedWindows( gabwFactory, windowingStrategy, "key", - WindowedValue.of( + WindowedValues.of( "v1", new Instant(1), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v2", new Instant(2), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v3", new Instant(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -154,12 +155,12 @@ public static void groupsElementsIntoSlidingWindowsWithMinTimestamp( gabwFactory, windowingStrategy, "key", - WindowedValue.of( + WindowedValues.of( "v1", new Instant(5), Arrays.asList(window(-10, 10), window(0, 20)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v2", new Instant(15), Arrays.asList(window(0, 20), window(10, 30)), @@ -203,17 +204,17 @@ public static void combinesElementsInSlidingWindows( gabwFactory, windowingStrategy, "k", - WindowedValue.of( + WindowedValues.of( 1L, new Instant(5), Arrays.asList(window(-10, 10), window(0, 20)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( 2L, new Instant(15), Arrays.asList(window(0, 20), window(10, 30)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( 4L, new Instant(18), Arrays.asList(window(0, 20), window(10, 30)), @@ -252,9 +253,11 @@ public static void groupsIntoOverlappingNonmergingWindows( gabwFactory, windowingStrategy, "key", - WindowedValue.of("v1", new Instant(1), Arrays.asList(window(0, 5)), PaneInfo.NO_FIRING), - WindowedValue.of("v2", new Instant(4), Arrays.asList(window(1, 5)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( + "v1", new Instant(1), Arrays.asList(window(0, 5)), PaneInfo.NO_FIRING), + WindowedValues.of( + "v2", new Instant(4), Arrays.asList(window(1, 5)), PaneInfo.NO_FIRING), + WindowedValues.of( "v3", new Instant(4), Arrays.asList(window(0, 5)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -282,11 +285,11 @@ public static void groupsElementsInMergedSessions( gabwFactory, windowingStrategy, "key", - WindowedValue.of( + WindowedValues.of( "v1", new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v2", new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v3", new Instant(15), Arrays.asList(window(15, 25)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -319,9 +322,9 @@ public static void combinesElementsPerSession( 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( + WindowedValues.of(1L, new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), + WindowedValues.of(2L, new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), + WindowedValues.of( 4L, new Instant(15), Arrays.asList(window(15, 25)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -354,11 +357,11 @@ public static void groupsElementsIntoFixedWindowsWithEndOfWindowTimestamp( gabwFactory, windowingStrategy, "key", - WindowedValue.of( + WindowedValues.of( "v1", new Instant(1), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v2", new Instant(2), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v3", new Instant(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -391,11 +394,11 @@ public static void groupsElementsIntoFixedWindowsWithLatestTimestamp( gabwFactory, windowingStrategy, "k", - WindowedValue.of( + WindowedValues.of( "v1", new Instant(1), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v2", new Instant(2), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v3", new Instant(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -427,11 +430,11 @@ public static void groupsElementsInMergedSessionsWithEndOfWindowTimestamp( gabwFactory, windowingStrategy, "k", - WindowedValue.of( + WindowedValues.of( "v1", new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v2", new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v3", new Instant(15), Arrays.asList(window(15, 25)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -464,11 +467,11 @@ public static void groupsElementsInMergedSessionsWithLatestTimestamp( gabwFactory, windowingStrategy, "k", - WindowedValue.of( + WindowedValues.of( "v1", new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v2", new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v3", new Instant(15), Arrays.asList(unmergedWindow), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -504,9 +507,10 @@ public static void combinesElementsPerSessionWithEndOfWindowTimestamp( 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(secondWindow), PaneInfo.NO_FIRING)); + WindowedValues.of(1L, new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), + WindowedValues.of(2L, new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), + WindowedValues.of( + 4L, new Instant(15), Arrays.asList(secondWindow), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -613,7 +617,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - this.output.add(WindowedValue.of(output, timestamp, windows, pane)); + this.output.add(WindowedValues.of(output, timestamp, windows, pane)); } public List>> getOutput() { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/ListOutputManager.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/ListOutputManager.java index 742ba68cf634..eccd6ec41666 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/ListOutputManager.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/ListOutputManager.java @@ -21,8 +21,8 @@ import java.util.List; import java.util.Map; import org.apache.beam.runners.core.DoFnRunners.OutputManager; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleCheckpointHandlers.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleCheckpointHandlers.java index c5ca5ae54361..14d00934a242 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleCheckpointHandlers.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleCheckpointHandlers.java @@ -32,7 +32,8 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Duration; import org.joda.time.Instant; @@ -127,7 +128,7 @@ public void onCheckpoint(ProcessBundleResponse response) { stateInternals .state(stateNamespace, StateTags.value(tag, residualCoder)) .write( - WindowedValue.of( + WindowedValues.of( stateValue.getValue(), stateValue.getTimestamp(), ImmutableList.of(window), diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java index 9d5589a54ede..aeb8aae81022 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java @@ -47,8 +47,6 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.state.TimerSpecs; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; @@ -58,6 +56,8 @@ import org.apache.beam.sdk.util.construction.graph.TimerReference; import org.apache.beam.sdk.util.construction.graph.UserStateReference; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactory.java index b26d3a643ef1..68f274f3de9c 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactory.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactory.java @@ -32,11 +32,11 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.sdk.util.construction.graph.PipelineNode.PCollectionNode; import org.apache.beam.sdk.util.construction.graph.SideInputReference; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.FluentIterable; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMultimap; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtils.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtils.java index 762a31749457..5b7880ad56ee 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtils.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtils.java @@ -36,12 +36,13 @@ import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.util.construction.graph.PipelineNode; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; @@ -169,15 +170,15 @@ private static void fireTimer( } } - public static WindowedValue.WindowedValueCoder getWindowedValueCoder( + public static WindowedValues.WindowedValueCoder getWindowedValueCoder( String pCollectionId, RunnerApi.Components components) { RunnerApi.PCollection pCollection = components.getPcollectionsOrThrow(pCollectionId); PipelineNode.PCollectionNode pCollectionNode = PipelineNode.pCollection(pCollectionId, pCollection); - WindowedValue.WindowedValueCoder coder; + WindowedValues.WindowedValueCoder coder; try { coder = - (WindowedValue.WindowedValueCoder) + (WindowedValues.WindowedValueCoder) WireCoders.instantiateRunnerWireCoder(pCollectionNode, components); } catch (IOException e) { throw new RuntimeException(e); diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java index f0dc2f65a0ea..0377d59a31fe 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java @@ -24,12 +24,13 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.WireCoderSetting; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.construction.ModelCoders; import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.apache.beam.sdk.util.construction.SyntheticComponents; import org.apache.beam.sdk.util.construction.graph.PipelineNode.PCollectionNode; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; /** Helpers to construct coders for gRPC port reads and writes. */ public class WireCoders { @@ -93,7 +94,7 @@ public static Coder> instantiateRunnerWireCoder( String protoCoderId = addRunnerWireCoder(pCollectionNode, builder, wireCoderSetting); Coder javaCoder = RehydratedComponents.forComponents(builder.build()).getCoder(protoCoderId); checkArgument( - javaCoder instanceof WindowedValue.FullWindowedValueCoder, + javaCoder instanceof WindowedValues.FullWindowedValueCoder, "Unexpected Deserialized %s type, expected %s, got %s", RunnerApi.Coder.class.getSimpleName(), FullWindowedValueCoder.class.getSimpleName(), diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java index b6b1b7be3355..556cc7993ae1 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.fnexecution.control; import static org.apache.beam.sdk.options.ExperimentalOptions.addExperiment; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.allOf; @@ -129,7 +129,6 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; @@ -142,6 +141,8 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @@ -1914,13 +1915,13 @@ public void onWindowExpiration( outputValues.get(mainOutputTransform), containsInAnyOrder( valueInGlobalWindow(KV.of("mainX", "")), - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of("event", "Y"), BoundedWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(100L))), - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of("processing", "Z"), BoundedWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(200L))), - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of("onWindowExpiration", "key"), BoundedWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(5000L))))); assertThat( diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java index 7ce6b53c5eb6..7b19376426cc 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java @@ -82,12 +82,13 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow.Coder; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.construction.CoderTranslation; import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -143,7 +144,7 @@ public void setup() throws Exception { .putAllWindowingStrategies(userProto.getComponents().getWindowingStrategiesMap()) .putAllCoders(userProto.getComponents().getCodersMap()); RunnerApi.Coder fullValueCoder = - CoderTranslation.toProto(WindowedValue.getFullCoder(StringUtf8Coder.of(), Coder.INSTANCE)) + CoderTranslation.toProto(WindowedValues.getFullCoder(StringUtf8Coder.of(), Coder.INSTANCE)) .getCoder(); pbdBuilder.putCoders("wire_coder", fullValueCoder); @@ -515,18 +516,18 @@ public void testNewBundleAndProcessElements() throws Exception { BundleProgressHandler.ignored())) { FnDataReceiver> bundleInputReceiver = Iterables.getOnlyElement(activeBundle.getInputReceivers().values()); - bundleInputReceiver.accept(WindowedValue.valueInGlobalWindow("foo")); - bundleInputReceiver.accept(WindowedValue.valueInGlobalWindow("bar")); - bundleInputReceiver.accept(WindowedValue.valueInGlobalWindow("baz")); + bundleInputReceiver.accept(WindowedValues.valueInGlobalWindow("foo")); + bundleInputReceiver.accept(WindowedValues.valueInGlobalWindow("bar")); + bundleInputReceiver.accept(WindowedValues.valueInGlobalWindow("baz")); } // The bundle can be a simple function of some sort, but needs to be complete. assertThat( outputs, containsInAnyOrder( - WindowedValue.valueInGlobalWindow("spam"), - WindowedValue.valueInGlobalWindow("ham"), - WindowedValue.valueInGlobalWindow("eggs"))); + WindowedValues.valueInGlobalWindow("spam"), + WindowedValues.valueInGlobalWindow("ham"), + WindowedValues.valueInGlobalWindow("eggs"))); } @Test diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/GrpcDataServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/GrpcDataServiceTest.java index 9f49309c3040..f84467077501 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/GrpcDataServiceTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/GrpcDataServiceTest.java @@ -49,7 +49,8 @@ import org.apache.beam.sdk.fn.stream.OutboundObserverFactory; import org.apache.beam.sdk.fn.test.TestStreams; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ManagedChannel; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.inprocess.InProcessChannelBuilder; @@ -66,7 +67,7 @@ public class GrpcDataServiceTest { @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final String TRANSFORM_ID = "888"; private static final Coder> CODER = - LengthPrefixCoder.of(WindowedValue.getValueOnlyCoder(StringUtf8Coder.of())); + LengthPrefixCoder.of(WindowedValues.getValueOnlyCoder(StringUtf8Coder.of())); @Test public void testMessageReceivedBySingleClientWhenThereAreMultipleClients() throws Exception { @@ -105,9 +106,9 @@ public void testMessageReceivedBySingleClientWhenThereAreMultipleClients() throw aggregator.start(); FnDataReceiver> consumer = aggregator.registerOutputDataLocation(TRANSFORM_ID, CODER); - consumer.accept(WindowedValue.valueInGlobalWindow("A" + i)); - consumer.accept(WindowedValue.valueInGlobalWindow("B" + i)); - consumer.accept(WindowedValue.valueInGlobalWindow("C" + i)); + consumer.accept(WindowedValues.valueInGlobalWindow("A" + i)); + consumer.accept(WindowedValues.valueInGlobalWindow("B" + i)); + consumer.accept(WindowedValues.valueInGlobalWindow("C" + i)); aggregator.sendOrCollectBufferedDataAndFinishOutboundStreams(); } waitForInboundElements.countDown(); @@ -175,9 +176,9 @@ public void testMultipleClientsSendMessagesAreDirectedToProperConsumers() throws assertThat( serverInboundValues.get(i), contains( - WindowedValue.valueInGlobalWindow("A" + i), - WindowedValue.valueInGlobalWindow("B" + i), - WindowedValue.valueInGlobalWindow("C" + i))); + WindowedValues.valueInGlobalWindow("A" + i), + WindowedValues.valueInGlobalWindow("B" + i), + WindowedValues.valueInGlobalWindow("C" + i))); } assertThat(clientInboundElements, empty()); } @@ -191,15 +192,15 @@ private BeamFnApi.Elements elementsWithData(String id) throws CoderException { .setTransformId(TRANSFORM_ID) .setData( ByteString.copyFrom( - encodeToByteArray(CODER, WindowedValue.valueInGlobalWindow("A" + id))) + encodeToByteArray(CODER, WindowedValues.valueInGlobalWindow("A" + id))) .concat( ByteString.copyFrom( encodeToByteArray( - CODER, WindowedValue.valueInGlobalWindow("B" + id)))) + CODER, WindowedValues.valueInGlobalWindow("B" + id)))) .concat( ByteString.copyFrom( encodeToByteArray( - CODER, WindowedValue.valueInGlobalWindow("C" + id)))))) + CODER, WindowedValues.valueInGlobalWindow("C" + id)))))) .addData( BeamFnApi.Elements.Data.newBuilder() .setInstructionId(id) diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java index f9a19dacc3df..70f9ef0dd4e1 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java @@ -41,13 +41,13 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.sdk.util.construction.graph.ImmutableExecutableStage; import org.apache.beam.sdk.util.construction.graph.PipelineNode; import org.apache.beam.sdk.util.construction.graph.PipelineNode.PCollectionNode; import org.apache.beam.sdk.util.construction.graph.SideInputReference; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValues; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Instant; @@ -121,7 +121,7 @@ public void emptyResultForEmptyCollection() { public void singleElementForCollection() { when(context.getSideInput(COLLECTION_ID)) .thenReturn( - Arrays.asList(WindowedValue.valueInGlobalWindow(KV.of(null, 3)))); + Arrays.asList(WindowedValues.valueInGlobalWindow(KV.of(null, 3)))); BatchSideInputHandlerFactory factory = BatchSideInputHandlerFactory.forStage(EXECUTABLE_STAGE, context); @@ -142,9 +142,9 @@ public void groupsValuesByKey() { when(context.getSideInput(COLLECTION_ID)) .thenReturn( Arrays.asList( - WindowedValue.valueInGlobalWindow(KV.of("foo", 2)), - WindowedValue.valueInGlobalWindow(KV.of("bar", 3)), - WindowedValue.valueInGlobalWindow(KV.of("foo", 5)))); + WindowedValues.valueInGlobalWindow(KV.of("foo", 2)), + WindowedValues.valueInGlobalWindow(KV.of("bar", 3)), + WindowedValues.valueInGlobalWindow(KV.of("foo", 5)))); BatchSideInputHandlerFactory factory = BatchSideInputHandlerFactory.forStage(EXECUTABLE_STAGE, context); @@ -170,12 +170,12 @@ public void groupsValuesByWindowAndKey() { when(context.getSideInput(COLLECTION_ID)) .thenReturn( Arrays.asList( - WindowedValue.of(KV.of("foo", 1), instantA, windowA, PaneInfo.NO_FIRING), - WindowedValue.of(KV.of("baz", 2), instantA, windowA, PaneInfo.NO_FIRING), - WindowedValue.of(KV.of("foo", 3), instantA, windowA, PaneInfo.NO_FIRING), - WindowedValue.of(KV.of("foo", 4), instantB, windowB, PaneInfo.NO_FIRING), - WindowedValue.of(KV.of("bar", 5), instantB, windowB, PaneInfo.NO_FIRING), - WindowedValue.of(KV.of("foo", 6), instantB, windowB, PaneInfo.NO_FIRING))); + WindowedValues.of(KV.of("foo", 1), instantA, windowA, PaneInfo.NO_FIRING), + WindowedValues.of(KV.of("baz", 2), instantA, windowA, PaneInfo.NO_FIRING), + WindowedValues.of(KV.of("foo", 3), instantA, windowA, PaneInfo.NO_FIRING), + WindowedValues.of(KV.of("foo", 4), instantB, windowB, PaneInfo.NO_FIRING), + WindowedValues.of(KV.of("bar", 5), instantB, windowB, PaneInfo.NO_FIRING), + WindowedValues.of(KV.of("foo", 6), instantB, windowB, PaneInfo.NO_FIRING))); BatchSideInputHandlerFactory factory = BatchSideInputHandlerFactory.forStage(EXECUTABLE_STAGE, context); @@ -206,10 +206,10 @@ public void iterableAccessPattern() { when(context.getSideInput(COLLECTION_ID)) .thenReturn( Arrays.asList( - WindowedValue.of(1, instantA, windowA, PaneInfo.NO_FIRING), - WindowedValue.of(2, instantA, windowA, PaneInfo.NO_FIRING), - WindowedValue.of(3, instantB, windowB, PaneInfo.NO_FIRING), - WindowedValue.of(4, instantB, windowB, PaneInfo.NO_FIRING))); + WindowedValues.of(1, instantA, windowA, PaneInfo.NO_FIRING), + WindowedValues.of(2, instantA, windowA, PaneInfo.NO_FIRING), + WindowedValues.of(3, instantB, windowB, PaneInfo.NO_FIRING), + WindowedValues.of(4, instantB, windowB, PaneInfo.NO_FIRING))); BatchSideInputHandlerFactory factory = BatchSideInputHandlerFactory.forStage(EXECUTABLE_STAGE, context); diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java index 2e24713eccbf..4f0e67286d62 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java @@ -80,13 +80,13 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.ShardedKey; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.util.construction.CoderTranslator; import org.apache.beam.sdk.util.construction.ModelCoderRegistrar; import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; @@ -131,10 +131,10 @@ public class CommonCoderTest { .put(getUrn(StandardCoders.Enum.DOUBLE), DoubleCoder.class) .put( getUrn(StandardCoders.Enum.WINDOWED_VALUE), - WindowedValue.FullWindowedValueCoder.class) + WindowedValues.FullWindowedValueCoder.class) .put( getUrn(StandardCoders.Enum.PARAM_WINDOWED_VALUE), - WindowedValue.ParamWindowedValueCoder.class) + WindowedValues.ParamWindowedValueCoder.class) .put(getUrn(StandardCoders.Enum.ROW), RowCoder.class) .put(getUrn(StandardCoders.Enum.SHARDED_KEY), ShardedKey.Coder.class) .put(getUrn(StandardCoders.Enum.CUSTOM_WINDOW), TimestampPrefixingWindowCoder.class) @@ -347,8 +347,8 @@ private static Object convertValue(Object value, CommonCoder coderSpec, Coder co } else if (s.equals(getUrn(StandardCoders.Enum.WINDOWED_VALUE)) || s.equals(getUrn(StandardCoders.Enum.PARAM_WINDOWED_VALUE))) { Map kvMap = (Map) value; - Coder valueCoder = ((WindowedValue.FullWindowedValueCoder) coder).getValueCoder(); - Coder windowCoder = ((WindowedValue.FullWindowedValueCoder) coder).getWindowCoder(); + Coder valueCoder = ((WindowedValues.FullWindowedValueCoder) coder).getValueCoder(); + Coder windowCoder = ((WindowedValues.FullWindowedValueCoder) coder).getWindowCoder(); Object windowValue = convertValue(kvMap.get("value"), coderSpec.getComponents().get(0), valueCoder); Instant timestamp = new Instant(((Number) kvMap.get("timestamp")).longValue()); @@ -365,7 +365,7 @@ private static Object convertValue(Object value, CommonCoder coderSpec, Coder co PaneInfo.Timing.valueOf((String) paneInfoMap.get("timing")), (int) paneInfoMap.get("index"), (int) paneInfoMap.get("on_time_index")); - return WindowedValue.of(windowValue, timestamp, windows, paneInfo); + return WindowedValues.of(windowValue, timestamp, windows, paneInfo); } else if (s.equals(getUrn(StandardCoders.Enum.DOUBLE))) { return Double.parseDouble((String) value); } else if (s.equals(getUrn(StandardCoders.Enum.ROW))) { diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java index 061c75284f85..ce44ad69dece 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java @@ -31,10 +31,10 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.LengthPrefixCoder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; import org.junit.Rule; @@ -77,10 +77,10 @@ public static Collection data() { return ImmutableList.of( /** Test wrapping unknown coders with {@code LengthPrefixCoder}. */ new Object[] { - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(UnknownCoder.INSTANCE, UnknownCoder.INSTANCE), GlobalWindow.Coder.INSTANCE), - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( LengthPrefixCoder.of(UnknownCoder.INSTANCE), LengthPrefixCoder.of(UnknownCoder.INSTANCE)), @@ -91,10 +91,10 @@ public static Collection data() { * Test bypassing unknown coders that are already wrapped with {@code LengthPrefixCoder}. */ new Object[] { - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(UnknownCoder.INSTANCE, LengthPrefixCoder.of(UnknownCoder.INSTANCE)), GlobalWindow.Coder.INSTANCE), - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( LengthPrefixCoder.of(UnknownCoder.INSTANCE), LengthPrefixCoder.of(UnknownCoder.INSTANCE)), @@ -103,10 +103,10 @@ public static Collection data() { }, /** Test replacing unknown coders with {@code LengthPrefixCoder}. */ new Object[] { - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(LengthPrefixCoder.of(UnknownCoder.INSTANCE), UnknownCoder.INSTANCE), GlobalWindow.Coder.INSTANCE), - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( LengthPrefixCoder.of(ByteArrayCoder.of()), LengthPrefixCoder.of(ByteArrayCoder.of())), diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/DAGBuilder.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/DAGBuilder.java index f7792e5e31ca..dd7585d6e9fa 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/DAGBuilder.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/DAGBuilder.java @@ -36,9 +36,10 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; /** Utility class for wiring up Jet DAGs based on Beam pipelines. */ @SuppressWarnings({ @@ -205,12 +206,12 @@ private int getNextFreeOrdinal(Vertex vertex, boolean inbound) { } private static class PartitionedKeyExtractor implements FunctionEx { - private final WindowedValue.WindowedValueCoder> coder; + private final WindowedValues.WindowedValueCoder> coder; PartitionedKeyExtractor(Coder coder) { this.coder = Utils.isKeyedValueCoder(coder) - ? (WindowedValue.WindowedValueCoder>) coder + ? (WindowedValues.WindowedValueCoder>) coder : null; } diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTransformTranslators.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTransformTranslators.java index f25b95fdb4f6..5ae3e1234b26 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTransformTranslators.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTransformTranslators.java @@ -46,7 +46,6 @@ import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CreatePCollectionViewTranslation; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.ParDoTranslation; @@ -59,6 +58,8 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; @SuppressWarnings({ @@ -269,7 +270,7 @@ public Vertex translate( PCollection> input = (PCollection>) Utils.getInput(appliedTransform); - WindowedValue.WindowedValueCoder> inputCoder = + WindowedValues.WindowedValueCoder> inputCoder = Utils.getWindowedValueCoder(input); Map.Entry, PCollection> output = Utils.getOutput(appliedTransform); Coder outputCoder = Utils.getCoder((PCollection) output.getValue()); diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTranslationContext.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTranslationContext.java index 08ac6dc91ee5..cd73fcb65f56 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTranslationContext.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTranslationContext.java @@ -19,8 +19,8 @@ import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; class JetTranslationContext { @@ -41,12 +41,12 @@ DAGBuilder getDagBuilder() { return dagBuilder; } - WindowedValue.FullWindowedValueCoder getTypeInfo(PCollection collection) { + WindowedValues.FullWindowedValueCoder getTypeInfo(PCollection collection) { return getTypeInfo(collection.getCoder(), collection.getWindowingStrategy()); } - WindowedValue.FullWindowedValueCoder getTypeInfo( + WindowedValues.FullWindowedValueCoder getTypeInfo( Coder coder, WindowingStrategy windowingStrategy) { - return WindowedValue.getFullCoder(coder, windowingStrategy.getWindowFn().windowCoder()); + return WindowedValues.getFullCoder(coder, windowingStrategy.getWindowFn().windowCoder()); } } diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/Utils.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/Utils.java index 38cb96b13a07..cb1cd69d33c6 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/Utils.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/Utils.java @@ -45,13 +45,14 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.checkerframework.checker.nullness.qual.Nullable; @@ -118,8 +119,8 @@ static boolean isBounded(AppliedPTransform appliedTransform) { static boolean isKeyedValueCoder(Coder coder) { if (coder instanceof KvCoder) { return true; - } else if (coder instanceof WindowedValue.WindowedValueCoder) { - return ((WindowedValue.WindowedValueCoder) coder).getValueCoder() instanceof KvCoder; + } else if (coder instanceof WindowedValues.WindowedValueCoder) { + return ((WindowedValues.WindowedValueCoder) coder).getValueCoder() instanceof KvCoder; } return false; } @@ -132,8 +133,9 @@ static Coder getCoder(PCollection pCollection) { } } - static WindowedValue.WindowedValueCoder getWindowedValueCoder(PCollection pCollection) { - return WindowedValue.FullWindowedValueCoder.of( + static WindowedValues.WindowedValueCoder getWindowedValueCoder( + PCollection pCollection) { + return WindowedValues.FullWindowedValueCoder.of( pCollection.getCoder(), pCollection.getWindowingStrategy().getWindowFn().windowCoder()); } @@ -260,9 +262,9 @@ public static WindowedValue decodeWindowedValue(byte[] item, Coder coder) } } - public static WindowedValue.FullWindowedValueCoder deriveIterableValueCoder( - WindowedValue.FullWindowedValueCoder elementCoder) { - return WindowedValue.FullWindowedValueCoder.of( + public static WindowedValues.FullWindowedValueCoder deriveIterableValueCoder( + WindowedValues.FullWindowedValueCoder elementCoder) { + return WindowedValues.FullWindowedValueCoder.of( ListCoder.of(elementCoder.getValueCoder()), elementCoder.getWindowCoder()); } diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AbstractParDoP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AbstractParDoP.java index c5bd4ecaf2ef..dcc4c39cd954 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AbstractParDoP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AbstractParDoP.java @@ -59,9 +59,10 @@ import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -132,7 +133,7 @@ abstract class AbstractParDoP implements Processor { Map.Entry::getKey, e -> Utils.deriveIterableValueCoder( - (WindowedValue.FullWindowedValueCoder) e.getValue()))); + (WindowedValues.FullWindowedValueCoder) e.getValue()))); this.outputCoders = outputCoders; this.inputValueCoder = inputValueCoder; this.outputValueCoders = outputValueCoders; diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AssignWindowP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AssignWindowP.java index 82151a812d63..da940a2b8cc7 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AssignWindowP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AssignWindowP.java @@ -27,7 +27,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.joda.time.Instant; @@ -73,7 +74,7 @@ private AssignWindowP( throw new RuntimeException(e); } WindowedValue outputValue = - WindowedValue.of( + WindowedValues.of( inputValue.getValue(), inputValue.getTimestamp(), windows, diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/BoundedSourceP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/BoundedSourceP.java index 4da880e618b1..5784ef308546 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/BoundedSourceP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/BoundedSourceP.java @@ -36,7 +36,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; /** * Jet {@link com.hazelcast.jet.core.Processor} implementation for reading from a bounded Beam @@ -79,7 +80,7 @@ public Object next() { try { Object item = currentReader.getCurrent(); WindowedValue res = - WindowedValue.timestampedValueInGlobalWindow(item, currentReader.getCurrentTimestamp()); + WindowedValues.timestampedValueInGlobalWindow(item, currentReader.getCurrentTimestamp()); if (!currentReader.advance()) { nextShard(); } diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/FlattenP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/FlattenP.java index 62648d2b4c64..f981eb40dfbd 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/FlattenP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/FlattenP.java @@ -27,7 +27,7 @@ import org.apache.beam.runners.jet.DAGBuilder; import org.apache.beam.runners.jet.Utils; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** Jet {@link com.hazelcast.jet.core.Processor} implementation for Beam's Flatten primitive. */ @SuppressWarnings({ diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ImpulseP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ImpulseP.java index 671dbd582baf..6775029d14fd 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ImpulseP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ImpulseP.java @@ -29,7 +29,7 @@ import javax.annotation.Nonnull; import org.apache.beam.runners.jet.Utils; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; /** * /** * Jet {@link com.hazelcast.jet.core.Processor} implementation for Beam's Impulse primitive. @@ -55,7 +55,7 @@ private ImpulseP(boolean active, Coder outputCoder, String ownerId) { @Override public boolean complete() { if (active) { - return tryEmit(Utils.encode(WindowedValue.valueInGlobalWindow(new byte[0]), outputCoder)); + return tryEmit(Utils.encode(WindowedValues.valueInGlobalWindow(new byte[0]), outputCoder)); } else { return true; } diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java index 3b3d79193622..f696873ec5c0 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java @@ -41,10 +41,10 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/UnboundedSourceP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/UnboundedSourceP.java index 38a75d47e703..4d426693f9b1 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/UnboundedSourceP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/UnboundedSourceP.java @@ -36,7 +36,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; /** * Jet {@link com.hazelcast.jet.core.Processor} implementation for reading from an unbounded Beam @@ -81,7 +81,7 @@ protected void init(@Nonnull Processor.Context context) throws IOException { Function, byte[]> mapFn = (reader) -> Utils.encode( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( reader.getCurrent(), reader.getCurrentTimestamp()), outputCoder); diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ViewP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ViewP.java index 1106585501ff..2ea7a998a896 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ViewP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ViewP.java @@ -33,7 +33,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; @@ -63,7 +64,7 @@ private ViewP( this.timestampCombiner = windowingStrategy.getTimestampCombiner(); this.inputCoder = inputCoder; this.outputCoder = - Utils.deriveIterableValueCoder((WindowedValue.FullWindowedValueCoder) outputCoder); + Utils.deriveIterableValueCoder((WindowedValues.FullWindowedValueCoder) outputCoder); this.ownerId = ownerId; } @@ -90,7 +91,7 @@ public boolean complete() { .map( e -> { WindowedValue outputValue = - WindowedValue.of( + WindowedValues.of( e.getValue().values, e.getValue().timestamp, Collections.singleton(e.getKey()), diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java index 81a9e8562fe0..4aad7091f2da 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java @@ -52,10 +52,11 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowTracing; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; @@ -95,7 +96,7 @@ public class WindowGroupP extends AbstractProcessor { private WindowGroupP( SerializablePipelineOptions pipelineOptions, - WindowedValue.WindowedValueCoder> inputCoder, + WindowedValues.WindowedValueCoder> inputCoder, Coder outputCoder, WindowingStrategy windowingStrategy, String ownerId) { @@ -130,7 +131,7 @@ private WindowGroupP( Utils.ByteArrayKey keyBytes = new Utils.ByteArrayKey(Utils.encode(key, inputValueCoder.getKeyCoder())); WindowedValue updatedWindowedValue = - WindowedValue.of( + WindowedValues.of( value, windowedValue.getTimestamp(), windowedValue.getWindows(), @@ -146,7 +147,7 @@ private WindowGroupP( @SuppressWarnings("unchecked") public static SupplierEx supplier( SerializablePipelineOptions pipelineOptions, - WindowedValue.WindowedValueCoder> inputCoder, + WindowedValues.WindowedValueCoder> inputCoder, Coder outputCoder, WindowingStrategy windowingStrategy, String ownerId) { @@ -234,7 +235,7 @@ public void outputWindowedValue( Collection windows, PaneInfo pane) { WindowedValue>> windowedValue = - WindowedValue.of(output, timestamp, windows, pane); + WindowedValues.of(output, timestamp, windows, pane); byte[] encodedValue = Utils.encode(windowedValue, outputCoder); //noinspection ResultOfMethodCallIgnored appendableTraverser.append(encodedValue); diff --git a/runners/jet/src/test/java/org/apache/beam/runners/jet/TestStreamP.java b/runners/jet/src/test/java/org/apache/beam/runners/jet/TestStreamP.java index 96b2614b1a71..291ca91b5c18 100644 --- a/runners/jet/src/test/java/org/apache/beam/runners/jet/TestStreamP.java +++ b/runners/jet/src/test/java/org/apache/beam/runners/jet/TestStreamP.java @@ -33,7 +33,7 @@ import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.joda.time.Instant; /** @@ -69,7 +69,7 @@ private TestStreamP(byte[] payload, TestStream.TestStreamCoder payloadCoder, Cod false) .map( tv -> - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( tv.getValue(), tv.getTimestamp())) .map(wV -> Utils.encode(wV, outputCoder)); } else { diff --git a/runners/local-java/src/main/java/org/apache/beam/runners/local/Bundle.java b/runners/local-java/src/main/java/org/apache/beam/runners/local/Bundle.java index c556db45c75b..7e0b42e561c3 100644 --- a/runners/local-java/src/main/java/org/apache/beam/runners/local/Bundle.java +++ b/runners/local-java/src/main/java/org/apache/beam/runners/local/Bundle.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.local; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java index eb2b423a1171..260957e05ecc 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java @@ -42,7 +42,8 @@ import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.samza.Partition; import org.apache.samza.SamzaException; @@ -314,7 +315,7 @@ private X invoke(FnWithMetricsWrapper.SupplierWithException fn) throws Ex private void enqueueMessage(BoundedReader reader) throws InterruptedException { final T value = reader.getCurrent(); final WindowedValue windowedValue = - WindowedValue.timestampedValueInGlobalWindow(value, reader.getCurrentTimestamp()); + WindowedValues.timestampedValueInGlobalWindow(value, reader.getCurrentTimestamp()); final SystemStreamPartition ssp = readerToSsp.get(reader); final IncomingMessageEnvelope envelope = new IncomingMessageEnvelope( diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java index 3b53894954d7..41214d7800e2 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java @@ -46,7 +46,8 @@ import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark; import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.samza.Partition; @@ -400,7 +401,7 @@ private void enqueueMessage(UnboundedReader reader) throws InterruptedException final Instant time = reader.getCurrentTimestamp(); final SystemStreamPartition ssp = readerToSsp.get(reader); final WindowedValue windowedValue = - WindowedValue.timestampedValueInGlobalWindow(value, time); + WindowedValues.timestampedValueInGlobalWindow(value, time); final OpMessage opMessage = OpMessage.ofElement(windowedValue); final IncomingMessageEnvelope envelope = diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/DoFnRunnerWithMetrics.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/DoFnRunnerWithMetrics.java index 7df33b531586..b815649a7659 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/DoFnRunnerWithMetrics.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/DoFnRunnerWithMetrics.java @@ -21,7 +21,7 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.joda.time.Instant; /** diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaGBKMetricOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaGBKMetricOp.java index 897c4a05b157..9b6375171c23 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaGBKMetricOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaGBKMetricOp.java @@ -26,7 +26,7 @@ import org.apache.beam.runners.samza.runtime.Op; import org.apache.beam.runners.samza.runtime.OpEmitter; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.samza.config.Config; import org.apache.samza.context.Context; import org.apache.samza.operators.Scheduler; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricOp.java index 8738d90c66c5..0f5334546c7c 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricOp.java @@ -27,7 +27,7 @@ import org.apache.beam.runners.samza.runtime.Op; import org.apache.beam.runners.samza.runtime.OpEmitter; import org.apache.beam.runners.samza.util.PipelineJsonRenderer; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.samza.config.Config; import org.apache.samza.context.Context; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunner.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunner.java index 76dfe5b720d8..d07a9bda78c6 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunner.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunner.java @@ -29,8 +29,8 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; import org.slf4j.Logger; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/ClassicBundleManager.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/ClassicBundleManager.java index 1d75afbb4c57..cf68cdb5b65c 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/ClassicBundleManager.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/ClassicBundleManager.java @@ -29,7 +29,7 @@ import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.samza.operators.Scheduler; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java index bb396ae0456b..11c1e8e3955a 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java @@ -52,11 +52,12 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; import org.apache.samza.config.Config; @@ -481,7 +482,7 @@ static CompletionStage> createOutputFuture( Function valueMapper) { return valueFuture.thenApply( res -> - WindowedValue.of( + WindowedValues.of( valueMapper.apply(res), windowedValue.getTimestamp(), windowedValue.getWindows(), diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnRunnerWithKeyedInternals.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnRunnerWithKeyedInternals.java index a07cf5fe2c8d..34e3405660c2 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnRunnerWithKeyedInternals.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnRunnerWithKeyedInternals.java @@ -22,8 +22,8 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.joda.time.Instant; /** This class wraps a DoFnRunner with keyed StateInternals and TimerInternals access. */ diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/FutureCollector.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/FutureCollector.java index c606b7569357..750e42d96e26 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/FutureCollector.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/FutureCollector.java @@ -19,7 +19,7 @@ import java.util.Collection; import java.util.concurrent.CompletionStage; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** * A future collector that buffers the output from the users {@link diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/FutureCollectorImpl.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/FutureCollectorImpl.java index f54f5e441b84..e364eb7c4078 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/FutureCollectorImpl.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/FutureCollectorImpl.java @@ -25,7 +25,7 @@ import java.util.concurrent.CompletionStage; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.beam.runners.samza.util.FutureUtils; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; class FutureCollectorImpl implements FutureCollector { private final AtomicBoolean collectorSealed; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/GroupByKeyOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/GroupByKeyOp.java index 1b19275dd967..b77a9de56a00 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/GroupByKeyOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/GroupByKeyOp.java @@ -40,10 +40,11 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.samza.config.Config; import org.apache.samza.context.Context; @@ -235,7 +236,7 @@ public void processTimer(KeyedTimerData keyedTimerData, OpEmitter implements Op, KeyedWorkItem, K> { diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/Op.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/Op.java index 93e6a9c2f613..14e0151bb708 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/Op.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/Op.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.samza.runtime; import java.io.Serializable; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.samza.config.Config; import org.apache.samza.context.Context; import org.apache.samza.operators.Scheduler; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpAdapter.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpAdapter.java index cbe81a1359e2..f2eecbbbc9c7 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpAdapter.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpAdapter.java @@ -34,7 +34,7 @@ import org.apache.beam.runners.samza.util.FutureUtils; import org.apache.beam.runners.samza.util.SamzaPipelineExceptionListener; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.samza.config.Config; import org.apache.samza.context.Context; import org.apache.samza.operators.Scheduler; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpEmitter.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpEmitter.java index cefbf0f8a2bf..c74d1cf1e11e 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpEmitter.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpEmitter.java @@ -19,7 +19,7 @@ import java.util.Collection; import java.util.concurrent.CompletionStage; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.joda.time.Instant; /** Output emitter for Samza {@link Op}. */ diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpMessage.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpMessage.java index dc288e6f901d..9ee7ffd48f2f 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpMessage.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpMessage.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.samza.runtime; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/PortableDoFnOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/PortableDoFnOp.java index f4e3641aeb9e..743a42d14791 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/PortableDoFnOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/PortableDoFnOp.java @@ -48,11 +48,11 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; import org.apache.samza.config.Config; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaAssignContext.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaAssignContext.java index 1a0caaa99c6d..0f8933e66cbf 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaAssignContext.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaAssignContext.java @@ -19,7 +19,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.joda.time.Instant; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java index 7625704de8ee..f1fd1b303044 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java @@ -57,7 +57,6 @@ import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.sdk.util.construction.graph.PipelineNode; @@ -65,6 +64,7 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.samza.context.Context; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SingletonKeyedWorkItem.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SingletonKeyedWorkItem.java index 7ec9906c5032..5a59e8616cc5 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SingletonKeyedWorkItem.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SingletonKeyedWorkItem.java @@ -20,7 +20,7 @@ import java.util.Collections; import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** Implementation of {@link KeyedWorkItem} which contains only a single value. */ class SingletonKeyedWorkItem implements KeyedWorkItem { diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SplittableParDoProcessKeyedElementsOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SplittableParDoProcessKeyedElementsOp.java index 2a6ba3e62a78..e8d8b64c381a 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SplittableParDoProcessKeyedElementsOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SplittableParDoProcessKeyedElementsOp.java @@ -44,11 +44,12 @@ import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.samza.config.Config; @@ -178,7 +179,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - outputManager.output(tag, WindowedValue.of(output, timestamp, windows, pane)); + outputManager.output(tag, WindowedValues.of(output, timestamp, windows, pane)); } }, NullSideInputReader.empty(), @@ -260,7 +261,7 @@ public void processTimer( private void fireTimer(byte[] key, TimerData timer) { LOG.debug("Firing timer {} for key {}", timer, key); fnRunner.processElement( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( KeyedWorkItems.timersWorkItem(key, Collections.singletonList(timer)))); } } diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/WindowAssignOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/WindowAssignOp.java index 48fb96917cb3..ed82589d2125 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/WindowAssignOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/WindowAssignOp.java @@ -20,7 +20,8 @@ import java.util.Collection; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; /** Samza operator for {@link org.apache.beam.sdk.transforms.windowing.Window.Assign}. */ public class WindowAssignOp implements Op { @@ -42,7 +43,7 @@ public void processElement(WindowedValue inputElement, OpEmitter emitter) windows.stream() .map( window -> - WindowedValue.of( + WindowedValues.of( inputElement.getValue(), inputElement.getTimestamp(), window, diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/GroupByKeyTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/GroupByKeyTranslator.java index 527522e66728..18d105e35e9f 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/GroupByKeyTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/GroupByKeyTranslator.java @@ -44,12 +44,13 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.AppliedCombineFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.graph.PipelineNode; import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.samza.operators.MessageStream; @@ -117,7 +118,7 @@ public void translatePortable( final MessageStream>> inputStream = ctx.getMessageStreamById(inputId); final WindowingStrategy windowingStrategy = WindowUtils.getWindowStrategy(inputId, pipeline.getComponents()); - final WindowedValue.WindowedValueCoder> windowedInputCoder = + final WindowedValues.WindowedValueCoder> windowedInputCoder = WindowUtils.instantiateWindowedCoder(inputId, pipeline.getComponents()); final TupleTag> outputTag = new TupleTag<>(Iterables.getOnlyElement(transform.getTransform().getOutputsMap().keySet())); @@ -151,14 +152,14 @@ static MessageStream>> doTranslate RunnerApi.PCollection input, MessageStream>> inputStream, WindowingStrategy windowingStrategy, - WindowedValue.WindowedValueCoder> windowedInputCoder, + WindowedValues.WindowedValueCoder> windowedInputCoder, TupleTag> outputTag, PortableTranslationContext ctx) { final boolean needRepartition = ctx.getPipelineOptions().getMaxSourceParallelism() > 1; final Coder windowCoder = windowingStrategy.getWindowFn().windowCoder(); final KvCoder kvInputCoder = (KvCoder) windowedInputCoder.getValueCoder(); final Coder>> elementCoder = - WindowedValue.FullWindowedValueCoder.of(kvInputCoder, windowCoder); + WindowedValues.FullWindowedValueCoder.of(kvInputCoder, windowCoder); @SuppressWarnings("unchecked") final SystemReduceFn reduceFn = diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java index e8668f5244b3..6b4d464b8e7c 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java @@ -56,7 +56,6 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.util.construction.RunnerPCollectionView; import org.apache.beam.sdk.util.construction.graph.PipelineNode; @@ -68,6 +67,8 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; import org.apache.samza.operators.MessageStream; @@ -262,8 +263,8 @@ private static void doTranslatePortable( .getInputsOrThrow(sideInputId.getLocalName()); final WindowingStrategy windowingStrategy = WindowUtils.getWindowStrategy(sideInputCollectionId, components); - final WindowedValue.WindowedValueCoder coder = - (WindowedValue.WindowedValueCoder) instantiateCoder(sideInputCollectionId, components); + final WindowedValues.WindowedValueCoder coder = + (WindowedValues.WindowedValueCoder) instantiateCoder(sideInputCollectionId, components); // Create a runner-side view final PCollectionView view = createPCollectionView(sideInputId, coder, windowingStrategy); @@ -305,7 +306,7 @@ private static void doTranslatePortable( index.incrementAndGet(); }); - WindowedValue.WindowedValueCoder windowedInputCoder = + WindowedValues.WindowedValueCoder windowedInputCoder = WindowUtils.instantiateWindowedCoder(inputId, pipeline.getComponents()); // TODO: support schema and side inputs for portable runner @@ -321,7 +322,7 @@ private static void doTranslatePortable( final Coder timerKeyCoder = stagePayload.getTimersCount() > 0 ? ((KvCoder) - ((WindowedValue.FullWindowedValueCoder) windowedInputCoder).getValueCoder()) + ((WindowedValues.FullWindowedValueCoder) windowedInputCoder).getValueCoder()) .getKeyCoder() : null; @@ -463,7 +464,7 @@ public Map createPortableConfig( // PCollectionView to represent a portable side input. private static PCollectionView createPCollectionView( SideInputId sideInputId, - WindowedValue.WindowedValueCoder coder, + WindowedValues.WindowedValueCoder coder, WindowingStrategy windowingStrategy) { return new RunnerPCollectionView<>( @@ -484,7 +485,7 @@ MessageStream>> groupAndBroadcastSideInput( String sideInputCollectionId, RunnerApi.PCollection sideInputPCollection, WindowingStrategy windowingStrategy, - WindowedValue.WindowedValueCoder coder, + WindowedValues.WindowedValueCoder coder, PortableTranslationContext ctx) { final MessageStream> sideInput = ctx.getMessageStreamById(sideInputCollectionId); @@ -494,7 +495,7 @@ MessageStream>> groupAndBroadcastSideInput( WindowedValue wv = opMessage.getElement(); return OpMessage.ofElement(wv.withValue(KV.of(null, wv.getValue()))); }); - final WindowedValue.WindowedValueCoder> kvCoder = + final WindowedValues.WindowedValueCoder> kvCoder = coder.withValueCoder(KvCoder.of(VoidCoder.of(), coder.getValueCoder())); final MessageStream>>> groupedSideInput = GroupByKeyTranslator.doTranslatePortable( diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReadTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReadTranslator.java index 59caecdc41c2..a666e36ad02f 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReadTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReadTranslator.java @@ -28,10 +28,10 @@ import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.samza.operators.KV; import org.apache.samza.serializers.KVSerde; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReshuffleTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReshuffleTranslator.java index bfcbd3edae40..c318505d9849 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReshuffleTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReshuffleTranslator.java @@ -26,12 +26,13 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.NativeTransforms; import org.apache.beam.sdk.util.construction.graph.PipelineNode; import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.samza.operators.MessageStream; import org.apache.samza.serializers.KVSerde; @@ -84,7 +85,7 @@ public void translatePortable( final String inputId = ctx.getInputId(transform); final MessageStream>> inputStream = ctx.getMessageStreamById(inputId); - final WindowedValue.WindowedValueCoder> windowedInputCoder = + final WindowedValues.WindowedValueCoder> windowedInputCoder = WindowUtils.instantiateWindowedCoder(inputId, pipeline.getComponents()); final String outputId = ctx.getOutputId(transform); diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaImpulseSystemFactory.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaImpulseSystemFactory.java index 7352e9a434a0..4035c1610e9f 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaImpulseSystemFactory.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaImpulseSystemFactory.java @@ -27,7 +27,7 @@ import java.util.stream.Collectors; import org.apache.beam.runners.samza.runtime.OpMessage; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.samza.Partition; import org.apache.samza.config.Config; import org.apache.samza.metrics.MetricsRegistry; @@ -118,7 +118,7 @@ private static List constructMessages(SystemStreamParti ssp, DUMMY_OFFSET, /* key */ null, - OpMessage.ofElement(WindowedValue.valueInGlobalWindow(new byte[0]))); + OpMessage.ofElement(WindowedValues.valueInGlobalWindow(new byte[0]))); final IncomingMessageEnvelope watermarkMessage = IncomingMessageEnvelope.buildWatermarkEnvelope( diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPublishViewTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPublishViewTranslator.java index 08b6196b6c43..08cf1057aabf 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPublishViewTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPublishViewTranslator.java @@ -23,8 +23,8 @@ import org.apache.beam.runners.samza.util.SamzaCoders; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.runners.TransformHierarchy; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.samza.operators.MessageStream; /** Translates {@link SamzaPublishView} to a view {@link MessageStream} as side input. */ diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamSystemFactory.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamSystemFactory.java index 7287994e9e8c..72cd711a6acc 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamSystemFactory.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamSystemFactory.java @@ -29,8 +29,9 @@ import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.samza.Partition; import org.apache.samza.SamzaException; @@ -144,7 +145,7 @@ public Map> poll( // timestamp. for (TimestampedValue element : ((TestStream.ElementEvent) event).getElements()) { WindowedValue windowedValue = - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( element.getValue(), element.getTimestamp()); final OpMessage opMessage = OpMessage.ofElement(windowedValue); final IncomingMessageEnvelope envelope = diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SplittableParDoTranslators.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SplittableParDoTranslators.java index 9a0e9a78da79..215c7e1fbcb7 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SplittableParDoTranslators.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SplittableParDoTranslators.java @@ -34,11 +34,11 @@ import org.apache.beam.sdk.runners.TransformHierarchy.Node; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.samza.operators.MessageStream; import org.apache.samza.serializers.KVSerde; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java index ae6015c5f00d..82725d1ce2e5 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java @@ -38,13 +38,14 @@ import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.samza.application.descriptors.StreamApplicationDescriptor; @@ -368,7 +369,7 @@ public StoreIdGenerator getStoreIdGenerator() { producer.send(id, new OutgoingMessageEnvelope(sysStream, 0, null, msg)); }; final WindowedValue windowedValue = - WindowedValue.timestampedValueInGlobalWindow("dummy", new Instant()); + WindowedValues.timestampedValueInGlobalWindow("dummy", new Instant()); sendFn.accept(OpMessage.ofElement(windowedValue)); sendFn.accept(new WatermarkMessage(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis())); diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaCoders.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaCoders.java index a118b5a704b5..f546303b9b9b 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaCoders.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaCoders.java @@ -22,8 +22,9 @@ import java.io.IOException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.samza.serializers.Serde; /** Utils for Coders creation/conversion in Samza. */ @@ -38,7 +39,7 @@ public static Coder> of(PCollection pCollection) { final Coder coder = pCollection.getCoder(); final Coder windowCoder = pCollection.getWindowingStrategy().getWindowFn().windowCoder(); - return WindowedValue.FullWindowedValueCoder.of(coder, windowCoder); + return WindowedValues.FullWindowedValueCoder.of(coder, windowCoder); } public static Serde toSerde(final Coder coder) { diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaPipelineTranslatorUtils.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaPipelineTranslatorUtils.java index 4ab1bcc8b0ee..999506e1e021 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaPipelineTranslatorUtils.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaPipelineTranslatorUtils.java @@ -20,9 +20,9 @@ import java.io.IOException; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.runners.fnexecution.wire.WireCoders; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.graph.PipelineNode; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValues; /** Utilities for pipeline translation. */ @SuppressWarnings({ @@ -31,12 +31,12 @@ public final class SamzaPipelineTranslatorUtils { private SamzaPipelineTranslatorUtils() {} - public static WindowedValue.WindowedValueCoder instantiateCoder( + public static WindowedValues.WindowedValueCoder instantiateCoder( String collectionId, RunnerApi.Components components) { PipelineNode.PCollectionNode collectionNode = PipelineNode.pCollection(collectionId, components.getPcollectionsOrThrow(collectionId)); try { - return (WindowedValue.WindowedValueCoder) + return (WindowedValues.WindowedValueCoder) WireCoders.instantiateRunnerWireCoder(collectionNode, components); } catch (IOException e) { throw new RuntimeException(e); diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/WindowUtils.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/WindowUtils.java index b4531d078933..7f39a9739c59 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/WindowUtils.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/WindowUtils.java @@ -21,10 +21,10 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.runners.fnexecution.wire.WireCoders; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.util.construction.graph.PipelineNode; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; /** Utils for window operations. */ @@ -57,15 +57,15 @@ public static WindowingStrategy getWindowStrategy( } /** - * Instantiate {@link WindowedValue.WindowedValueCoder} for given collection id from {@link + * Instantiate {@link WindowedValues.WindowedValueCoder} for given collection id from {@link * RunnerApi.Components}. */ - public static WindowedValue.WindowedValueCoder instantiateWindowedCoder( + public static WindowedValues.WindowedValueCoder instantiateWindowedCoder( String collectionId, RunnerApi.Components components) { PipelineNode.PCollectionNode collectionNode = PipelineNode.pCollection(collectionId, components.getPcollectionsOrThrow(collectionId)); try { - return (WindowedValue.WindowedValueCoder) + return (WindowedValues.WindowedValueCoder) WireCoders.instantiateRunnerWireCoder(collectionNode, components); } catch (IOException e) { throw new RuntimeException(e); diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/TestSourceHelpers.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/TestSourceHelpers.java index b020a9acd6ce..60c0c27b8d1f 100644 --- a/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/TestSourceHelpers.java +++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/TestSourceHelpers.java @@ -29,7 +29,7 @@ import org.apache.beam.runners.samza.runtime.OpMessage; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.samza.system.IncomingMessageEnvelope; import org.apache.samza.system.SystemStreamPartition; import org.joda.time.Instant; @@ -131,7 +131,7 @@ static IncomingMessageEnvelope createElementMessage( ssp, offset, null, - OpMessage.ofElement(WindowedValue.timestampedValueInGlobalWindow(element, timestamp))); + OpMessage.ofElement(WindowedValues.timestampedValueInGlobalWindow(element, timestamp))); } static IncomingMessageEnvelope createWatermarkMessage( diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/metrics/TestSamzaRunnerWithTransformMetrics.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/metrics/TestSamzaRunnerWithTransformMetrics.java index 68674d202cdb..dc660ce85af5 100644 --- a/runners/samza/src/test/java/org/apache/beam/runners/samza/metrics/TestSamzaRunnerWithTransformMetrics.java +++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/metrics/TestSamzaRunnerWithTransformMetrics.java @@ -43,9 +43,10 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.samza.context.Context; import org.apache.samza.metrics.Counter; @@ -175,9 +176,9 @@ public void testSamzaRunnerWithDefaultMetrics() { @Test public void testSamzaInputAndOutputMetricOp() { final WindowedValue windowedValue = - WindowedValue.timestampedValueInGlobalWindow("value-1", new Instant()); + WindowedValues.timestampedValueInGlobalWindow("value-1", new Instant()); final WindowedValue windowedValue2 = - WindowedValue.timestampedValueInGlobalWindow("value-2", new Instant()); + WindowedValues.timestampedValueInGlobalWindow("value-2", new Instant()); final WatermarkMessage watermarkMessage = new WatermarkMessage(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()); @@ -250,9 +251,9 @@ public void testSamzaInputAndOutputMetricOp() { @Test public void testSamzaInputAndOutputGBKMetricOp() { final WindowedValue windowedValue = - WindowedValue.timestampedValueInGlobalWindow("value-1", new Instant()); + WindowedValues.timestampedValueInGlobalWindow("value-1", new Instant()); final WindowedValue windowedValue2 = - WindowedValue.timestampedValueInGlobalWindow("value-2", new Instant()); + WindowedValues.timestampedValueInGlobalWindow("value-2", new Instant()); final WatermarkMessage watermarkMessage = new WatermarkMessage(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()); diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunnerTest.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunnerTest.java index f6f88eb8b280..4040040d3e04 100644 --- a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunnerTest.java +++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunnerTest.java @@ -46,10 +46,11 @@ import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Sum; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.Ignore; import org.junit.Rule; @@ -214,10 +215,10 @@ public void testKeyedOutputFutures() { final String appleKey = "apple"; final WindowedValue> input1 = - WindowedValue.valueInGlobalWindow(KV.of(appleKey, 1)); + WindowedValues.valueInGlobalWindow(KV.of(appleKey, 1)); final WindowedValue> input2 = - WindowedValue.valueInGlobalWindow(KV.of(appleKey, 2)); + WindowedValues.valueInGlobalWindow(KV.of(appleKey, 2)); asyncDoFnRunner.processElement(input1); asyncDoFnRunner.processElement(input2); diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/ClassicBundleManagerTest.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/ClassicBundleManagerTest.java index 1072e628a70c..8ff29ecf3838 100644 --- a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/ClassicBundleManagerTest.java +++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/ClassicBundleManagerTest.java @@ -35,7 +35,7 @@ import java.util.concurrent.CountDownLatch; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.samza.operators.Scheduler; import org.joda.time.Instant; import org.junit.Before; diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/FutureCollectorImplTest.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/FutureCollectorImplTest.java index a808fd223960..b0b9b5450f23 100644 --- a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/FutureCollectorImplTest.java +++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/FutureCollectorImplTest.java @@ -25,7 +25,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import java.util.stream.Collectors; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Assert; import org.junit.Before; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/io/BoundedDatasetFactory.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/io/BoundedDatasetFactory.java index c290a40287a5..c0d46e77c1d6 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/io/BoundedDatasetFactory.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/io/BoundedDatasetFactory.java @@ -19,7 +19,7 @@ import static java.util.stream.Collectors.toList; import static org.apache.beam.runners.spark.structuredstreaming.translation.utils.ScalaInterop.emptyList; -import static org.apache.beam.sdk.util.WindowedValue.timestampedValueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.timestampedValueInGlobalWindow; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static scala.collection.JavaConverters.asScalaIterator; @@ -36,7 +36,7 @@ import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.AbstractIterator; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.spark.InterruptibleIterator; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/EvaluationContext.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/EvaluationContext.java index da89ece0a2f7..32cbe5b0acab 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/EvaluationContext.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/EvaluationContext.java @@ -20,7 +20,7 @@ import java.util.Collection; import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Throwables; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.SparkSession; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java index 71325208c5ad..a681dea2fde5 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java @@ -46,13 +46,13 @@ import org.apache.beam.sdk.runners.TransformHierarchy.Node; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.View; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.spark.broadcast.Broadcast; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoder; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java index 13cc22940263..5b9e5b6fae86 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java @@ -74,11 +74,11 @@ import org.apache.beam.sdk.transforms.join.UnionCoder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.resources.PipelineResources; import org.apache.beam.sdk.values.PCollectionViews; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Collections2; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.spark.SparkConf; @@ -269,9 +269,9 @@ public void registerClasses(Kryo kryo) { // bounded windows and windowed value coders kryo.register(GlobalWindow.Coder.class); kryo.register(IntervalWindow.IntervalWindowCoder.class); - kryo.register(WindowedValue.FullWindowedValueCoder.class); - kryo.register(WindowedValue.ParamWindowedValueCoder.class); - kryo.register(WindowedValue.ValueOnlyWindowedValueCoder.class); + kryo.register(WindowedValues.FullWindowedValueCoder.class); + kryo.register(WindowedValues.ParamWindowedValueCoder.class); + kryo.register(WindowedValues.ValueOnlyWindowedValueCoder.class); // various others kryo.register(OffsetRange.Coder.class); diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TransformTranslator.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TransformTranslator.java index 2a01af5fb76e..24783040704e 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TransformTranslator.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TransformTranslator.java @@ -34,12 +34,12 @@ import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.spark.broadcast.Broadcast; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoder; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/Aggregators.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/Aggregators.java index 24d0f2eced69..183445642a0b 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/Aggregators.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/Aggregators.java @@ -43,7 +43,8 @@ import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Collections2; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -534,7 +535,7 @@ public final Collection> finish(MapT buffer) { } private WindowedValue windowedValue(Entry> e) { - return WindowedValue.of(extract(e.getValue()._2), e.getValue()._1, e.getKey(), NO_FIRING); + return WindowedValues.of(extract(e.getValue()._2), e.getValue()._1, e.getKey(), NO_FIRING); } } diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineGloballyTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineGloballyTranslatorBatch.java index c2bc05bed413..c77637ab5b91 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineGloballyTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineGloballyTranslatorBatch.java @@ -30,8 +30,9 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoder; @@ -120,6 +121,6 @@ private Coder accumulatorCoder( } private static Fun1>> windowedValue() { - return v -> single(WindowedValue.valueInGlobalWindow(v)); + return v -> single(WindowedValues.valueInGlobalWindow(v)); } } diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineGroupedValuesTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineGroupedValuesTranslatorBatch.java index cf9a2faac9cb..fa59cdf2452c 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineGroupedValuesTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineGroupedValuesTranslatorBatch.java @@ -23,9 +23,9 @@ import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.CombineWithContext; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoder; import org.apache.spark.sql.expressions.Aggregator; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombinePerKeyTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombinePerKeyTranslatorBatch.java index 63f3da0bd3dc..02c56a8081cf 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombinePerKeyTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombinePerKeyTranslatorBatch.java @@ -37,9 +37,10 @@ import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoder; @@ -146,7 +147,7 @@ public void translate(Combine.PerKey transform, Context cxt) { } private static Fun1, WindowedValue>> globalKV() { - return t -> WindowedValue.valueInGlobalWindow(KV.of(t._1, t._2)); + return t -> WindowedValues.valueInGlobalWindow(KV.of(t._1, t._2)); } private Encoder accumEncoder( diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnPartitionIteratorFactory.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnPartitionIteratorFactory.java index 907b847c7e8a..8d289f6b870c 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnPartitionIteratorFactory.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnPartitionIteratorFactory.java @@ -30,8 +30,8 @@ import org.apache.beam.runners.spark.structuredstreaming.metrics.MetricsAccumulator; import org.apache.beam.runners.spark.structuredstreaming.translation.batch.DoFnRunnerFactory.DoFnRunnerWithTeardown; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.AbstractIterator; import org.checkerframework.checker.nullness.qual.NonNull; import scala.Function1; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java index 6b548eff88c2..a2f81019910f 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java @@ -37,11 +37,11 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java index 6b7e353b164a..db4ee5be5786 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java @@ -28,7 +28,7 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.joda.time.Instant; /** DoFnRunner decorator which registers {@link MetricsContainer}. */ diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTranslatorBatch.java index fd02a3ef5242..63786829bd53 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTranslatorBatch.java @@ -24,9 +24,9 @@ import org.apache.beam.runners.spark.structuredstreaming.translation.TransformTranslator; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.Flatten; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoder; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyHelpers.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyHelpers.java index 28ab07114c6a..b7c139068d1b 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyHelpers.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyHelpers.java @@ -27,8 +27,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import scala.Tuple2; import scala.collection.TraversableOnce; @@ -89,7 +90,7 @@ static Fun1, V>, WindowedValue>> } static WindowedValue> windowedKV(Tuple2 key, V value) { - return WindowedValue.of(KV.of(key._2, value), key._1.maxTimestamp(), key._1, NO_FIRING); + return WindowedValues.of(KV.of(key._2, value), key._1.maxTimestamp(), key._1, NO_FIRING); } static Fun1, V> value() { diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTranslatorBatch.java index bbda0f0cfb65..7ab70f3652c8 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTranslatorBatch.java @@ -58,9 +58,10 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.spark.sql.Column; import org.apache.spark.sql.Dataset; @@ -155,7 +156,7 @@ public void translate(GroupByKey transform, Context cxt) { .groupByKey(valueKey(), keyEnc) .mapValues(valueValue(), cxt.valueEncoderOf(inputCoder)) .mapGroups(fun2((k, it) -> KV.of(k, iterableOnce(it))), cxt.kvEncoderOf(outputCoder)) - .map(fun1(WindowedValue::valueInGlobalWindow), cxt.windowedEncoder(outputCoder)); + .map(fun1(WindowedValues::valueInGlobalWindow), cxt.windowedEncoder(outputCoder)); } else if (useCollectList && eligibleForGroupByWindow(windowing, false) diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ImpulseTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ImpulseTranslatorBatch.java index 28f929679dd6..78afdfa5451e 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ImpulseTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ImpulseTranslatorBatch.java @@ -23,9 +23,10 @@ import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.transforms.Impulse; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.spark.sql.Dataset; @@ -39,7 +40,7 @@ class ImpulseTranslatorBatch extends TransformTranslator> dataset = cxt.createDataset( - ImmutableList.of(WindowedValue.valueInGlobalWindow(EMPTY_BYTE_ARRAY)), + ImmutableList.of(WindowedValues.valueInGlobalWindow(EMPTY_BYTE_ARRAY)), cxt.windowedEncoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE)); cxt.putDataset(cxt.getOutput(), dataset); } diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTranslatorBatch.java index c212f377eef5..0f43f329b0df 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTranslatorBatch.java @@ -45,12 +45,12 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; import org.apache.spark.broadcast.Broadcast; import org.apache.spark.sql.Dataset; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReadSourceTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReadSourceTranslatorBatch.java index e395c3f35908..e83ada473d0c 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReadSourceTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReadSourceTranslatorBatch.java @@ -24,10 +24,10 @@ import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.spark.sql.Encoder; import org.apache.spark.sql.SparkSession; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReshuffleTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReshuffleTranslatorBatch.java index e2bb27a750f1..2c541ba4ae43 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReshuffleTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReshuffleTranslatorBatch.java @@ -22,9 +22,9 @@ import java.io.IOException; import org.apache.beam.runners.spark.structuredstreaming.translation.TransformTranslator; import org.apache.beam.sdk.transforms.Reshuffle; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.internal.SQLConf; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTranslatorBatch.java index 140f2f88d8cd..adac966f457d 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTranslatorBatch.java @@ -23,8 +23,9 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.Dataset; @@ -93,7 +94,7 @@ public T element() { return window; } }); - return WindowedValue.of(element, timestamp, windows, value.getPane()); + return WindowedValues.of(element, timestamp, windows, value.getPane()); }; } diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/GroupAlsoByWindowViaOutputBufferFn.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/GroupAlsoByWindowViaOutputBufferFn.java index b348eca0c436..61c568dc2585 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/GroupAlsoByWindowViaOutputBufferFn.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/GroupAlsoByWindowViaOutputBufferFn.java @@ -35,10 +35,11 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.spark.api.java.function.FlatMapGroupsFunction; import org.joda.time.Instant; @@ -154,7 +155,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - outputs.add(WindowedValue.of(output, timestamp, windows, pane)); + outputs.add(WindowedValues.of(output, timestamp, windows, pane)); } @Override diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SideInputValues.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SideInputValues.java index 57d97cff1625..23c8d49c3091 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SideInputValues.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SideInputValues.java @@ -19,7 +19,7 @@ import static org.apache.beam.runners.spark.structuredstreaming.translation.utils.ScalaInterop.fun1; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; -import static org.apache.beam.sdk.util.WindowedValue.getFullCoder; +import static org.apache.beam.sdk.values.WindowedValues.getFullCoder; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.spark.sql.Encoders.BINARY; @@ -42,8 +42,8 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.spark.sql.Dataset; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java index 7a19c1c0e027..55f22f92a132 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java @@ -45,8 +45,9 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -354,7 +355,7 @@ private static Expression deserializeWindowedValue( ifNotNull(timestamp, invoke(Utils.class, "maxTimestamp", timestamp.dataType(), windows)); Expression[] fields = new Expression[] {value, timestamp, windows, pane}; - return nullSafe(pane, invoke(WindowedValue.class, "of", WINDOWED_VALUE, fields)); + return nullSafe(pane, invoke(WindowedValues.class, "of", WINDOWED_VALUE, fields)); } private static Expression serializeMutablePair( diff --git a/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/AggregatorsTest.java b/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/AggregatorsTest.java index d642a6e9866c..fa5312684fc1 100644 --- a/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/AggregatorsTest.java +++ b/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/AggregatorsTest.java @@ -39,8 +39,9 @@ import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Streams; import org.apache.spark.sql.Encoder; @@ -124,7 +125,7 @@ public void testMerge() { } private WindowedValue windowedValue(Integer value, Instant ts) { - return WindowedValue.of(value, ts, sliding.assignWindows(ts), PaneInfo.NO_FIRING); + return WindowedValues.of(value, ts, sliding.assignWindows(ts), PaneInfo.NO_FIRING); } } @@ -235,7 +236,7 @@ public void testMerge() { } private WindowedValue sessionValue(Integer value, Instant ts) { - return WindowedValue.of(value, ts, new IntervalWindow(ts, SESSIONS_GAP), PaneInfo.NO_FIRING); + return WindowedValues.of(value, ts, new IntervalWindow(ts, SESSIONS_GAP), PaneInfo.NO_FIRING); } private IntervalWindow sessionWindow(int fromMinutes) { diff --git a/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SideInputValuesTest.java b/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SideInputValuesTest.java index 3b828ee7d181..74d5a0292edb 100644 --- a/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SideInputValuesTest.java +++ b/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SideInputValuesTest.java @@ -21,8 +21,8 @@ import static org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderHelpers.windowedValueEncoder; import static org.apache.beam.runners.spark.structuredstreaming.translation.utils.ScalaInterop.seqOf; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; -import static org.apache.beam.sdk.util.WindowedValue.getFullCoder; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.getFullCoder; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.assertj.core.api.Assertions.assertThat; import com.esotericsoftware.kryo.Kryo; @@ -36,7 +36,8 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.spark.serializer.KryoSerializer; import org.apache.spark.sql.Dataset; @@ -102,7 +103,7 @@ private static IntervalWindow intervalWindow(int start, int end) { } private static WindowedValue valueInWindows(T value, BoundedWindow... windows) { - return WindowedValue.of(value, Instant.EPOCH, Lists.list(windows), PaneInfo.NO_FIRING); + return WindowedValues.of(value, Instant.EPOCH, Lists.list(windows), PaneInfo.NO_FIRING); } public static class SparkKryo extends ExternalResource { diff --git a/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpersTest.java b/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpersTest.java index ecb79d97b3e2..11c47ad03cb3 100644 --- a/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpersTest.java +++ b/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpersTest.java @@ -64,9 +64,10 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -139,7 +140,7 @@ public void testBeamWindowedValueEncoderMappings() { BASIC_CASES.forEach( (coder, data) -> { List> windowed = - Lists.transform(data, WindowedValue::valueInGlobalWindow); + Lists.transform(data, WindowedValues::valueInGlobalWindow); Encoder encoder = windowedValueEncoder(encoderFor(coder), windowEnc); serializeAndDeserialize(windowed.get(0), (Encoder) encoder); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/SparkRunnerKryoRegistrator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/SparkRunnerKryoRegistrator.java index 44f8d6df683b..68c602ff7f59 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/SparkRunnerKryoRegistrator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/SparkRunnerKryoRegistrator.java @@ -65,7 +65,8 @@ public void registerClasses(Kryo kryo) { try { kryo.register( - Class.forName("org.apache.beam.sdk.util.WindowedValue$TimestampedValueInGlobalWindow")); + Class.forName( + "org.apache.beam.sdk.values.WindowedValues$TimestampedValueInGlobalWindow")); kryo.register( Class.forName( "org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashBasedTable$Factory")); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java index c846ac20a626..e65dccd23f24 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java @@ -35,7 +35,8 @@ import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.metrics.MetricsContainer; import org.apache.beam.sdk.metrics.MetricsEnvironment; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.spark.Dependency; import org.apache.spark.HashPartitioner; @@ -189,7 +190,7 @@ private boolean tryProduceNext() { checkState(next == null, "unexpected non-null value for next"); if (seekNext()) { next = - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( reader.getCurrent(), reader.getCurrentTimestamp()); return SUCCESSFULLY_OBTAINED_NEXT; } else { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java index 34ef3331ae49..bea1557a7103 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java @@ -41,7 +41,8 @@ import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext$; @@ -120,8 +121,8 @@ public static UnboundedDataset re .register(); // output the actual (deserialized) stream. - WindowedValue.FullWindowedValueCoder coder = - WindowedValue.FullWindowedValueCoder.of( + WindowedValues.FullWindowedValueCoder coder = + WindowedValues.FullWindowedValueCoder.of( source.getOutputCoder(), GlobalWindow.Coder.INSTANCE); JavaDStream> readUnboundedStream = mapWithStateDStream diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java index b18b31a67463..1b9290ca7175 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java @@ -51,11 +51,12 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Predicate; @@ -110,7 +111,7 @@ public void outputWindowedValue( final Instant timestamp, final Collection windows, final PaneInfo pane) { - windowedValues.add(WindowedValue.of(output, timestamp, windows, pane)); + windowedValues.add(WindowedValues.of(output, timestamp, windows, pane)); } private List>>> getWindowedValues() { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java index b72481128f53..f995b47876fd 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java @@ -37,7 +37,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Stopwatch; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; @@ -164,15 +165,15 @@ public Tuple2, Metadata> apply( // read microbatch as a serialized collection. final List readValues = new ArrayList<>(); - WindowedValue.FullWindowedValueCoder coder = - WindowedValue.FullWindowedValueCoder.of( + WindowedValues.FullWindowedValueCoder coder = + WindowedValues.FullWindowedValueCoder.of( source.getOutputCoder(), GlobalWindow.Coder.INSTANCE); try { // measure how long a read takes per-partition. boolean finished = !microbatchReader.start(); while (!finished) { final WindowedValue wv = - WindowedValue.of( + WindowedValues.of( (T) microbatchReader.getCurrent(), microbatchReader.getCurrentTimestamp(), GlobalWindow.INSTANCE, diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java index 5cebc513e8b9..00e408cd8477 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java @@ -24,8 +24,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDDLike; @@ -56,7 +57,7 @@ public class BoundedDataset implements Dataset { } BoundedDataset(Iterable values, JavaSparkContext jsc, Coder coder) { - this.windowedValues = Iterables.transform(values, WindowedValue::valueInGlobalWindow); + this.windowedValues = Iterables.transform(values, WindowedValues::valueInGlobalWindow); this.jsc = jsc; this.coder = coder; } @@ -64,8 +65,8 @@ public class BoundedDataset implements Dataset { @SuppressWarnings("ConstantConditions") public JavaRDD> getRDD() { if (rdd == null) { - WindowedValue.ValueOnlyWindowedValueCoder windowCoder = - WindowedValue.getValueOnlyCoder(coder); + WindowedValues.ValueOnlyWindowedValueCoder windowCoder = + WindowedValues.getValueOnlyCoder(coder); rdd = jsc.parallelize(CoderHelpers.toByteArrays(windowedValues, windowCoder)) .map(CoderHelpers.fromByteFunction(windowCoder)); @@ -73,7 +74,7 @@ public JavaRDD> getRDD() { return rdd; } - List getBytes(WindowedValue.WindowedValueCoder wvCoder) { + List getBytes(WindowedValues.WindowedValueCoder wvCoder) { if (clientBytes == null) { JavaRDDLike bytesRDD = rdd.map(CoderHelpers.toByteFunction(wvCoder)); clientBytes = bytesRDD.collect(); @@ -85,12 +86,12 @@ Iterable> getValues(PCollection pcollection) { if (windowedValues == null) { WindowFn windowFn = pcollection.getWindowingStrategy().getWindowFn(); Coder windowCoder = windowFn.windowCoder(); - final WindowedValue.WindowedValueCoder windowedValueCoder; + final WindowedValues.WindowedValueCoder windowedValueCoder; if (windowFn instanceof GlobalWindows) { - windowedValueCoder = WindowedValue.ValueOnlyWindowedValueCoder.of(pcollection.getCoder()); + windowedValueCoder = WindowedValues.ValueOnlyWindowedValueCoder.of(pcollection.getCoder()); } else { windowedValueCoder = - WindowedValue.FullWindowedValueCoder.of(pcollection.getCoder(), windowCoder); + WindowedValues.FullWindowedValueCoder.of(pcollection.getCoder(), windowCoder); } JavaRDDLike bytesRDD = rdd.map(CoderHelpers.toByteFunction(windowedValueCoder)); List clientBytes = bytesRDD.collect(); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java index 34836cd6e7ae..a6b1f65571dc 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java @@ -27,7 +27,7 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.joda.time.Instant; /** DoFnRunner decorator which registers {@link MetricsContainerImpl}. */ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java index 99f2a3e4c360..8adc78af59ef 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java @@ -35,12 +35,13 @@ import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -208,7 +209,7 @@ private void putDataset( Coder coder = ((PCollection) pvalue).getCoder(); Coder wCoder = ((PCollection) pvalue).getWindowingStrategy().getWindowFn().windowCoder(); - dataset.cache(storageLevel(), WindowedValue.getFullCoder(coder, wCoder)); + dataset.cache(storageLevel(), WindowedValues.getFullCoder(coder, wCoder)); } datasets.put(pvalue, dataset); leaves.add(dataset); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java index 1d8901ed5ffc..03f7885f7e9f 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java @@ -25,9 +25,9 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.Reshuffle; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java index 14630fbb0a1f..3944cde5bee3 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java @@ -30,9 +30,10 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.AbstractIterator; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -87,7 +88,7 @@ JavaRDD>>> groupByKeyAndWindow( Partitioner partitioner) { final Coder windowCoder = windowingStrategy.getWindowFn().windowCoder(); FullWindowedValueCoder> windowedKvCoder = - WindowedValue.FullWindowedValueCoder.of(KvCoder.of(keyCoder, valueCoder), windowCoder); + WindowedValues.FullWindowedValueCoder.of(KvCoder.of(keyCoder, valueCoder), windowCoder); JavaPairRDD windowInKey = bringWindowToKey( rdd, keyCoder, windowCoder, wv -> CoderHelpers.toByteArray(wv, windowedKvCoder)); @@ -131,7 +132,7 @@ static JavaPairRDD final W window = (W) Iterables.getOnlyElement(item.getWindows()); final byte[] windowBytes = CoderHelpers.toByteArray(window, windowCoder); WindowedValue> valueOut = - WindowedValue.of(item.getValue(), item.getTimestamp(), window, item.getPane()); + WindowedValues.of(item.getValue(), item.getTimestamp(), window, item.getPane()); final ByteArray windowedKey = new ByteArray(Bytes.concat(keyBytes, windowBytes)); return new Tuple2<>(windowedKey, mappingFn.apply(valueOut)); }); @@ -179,7 +180,7 @@ static class GroupByKeyIterator Iterator> inner, Coder keyCoder, WindowingStrategy windowingStrategy, - WindowedValue.FullWindowedValueCoder> windowedValueCoder) + WindowedValues.FullWindowedValueCoder> windowedValueCoder) throws Coder.NonDeterministicException { this.inner = Iterators.peekingIterator(inner); @@ -258,7 +259,7 @@ private WindowedValue> decodeItem(Tuple2 item) { final Instant timestamp = windowingStrategy.getTimestampCombiner().assign(window, windowedValue.getTimestamp()); // BEAM-7341: Elements produced by GbK are always ON_TIME and ONLY_FIRING - return WindowedValue.of( + return WindowedValues.of( KV.of(key, value), timestamp, window, PaneInfo.ON_TIME_AND_ONLY_FIRING); } } @@ -297,7 +298,7 @@ JavaRDD>>> groupByKeyInGlobalWindow( Iterators.transform( iter, kvs -> - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of( CoderHelpers.fromByteArray(kvs._1.getValue(), keyCoder), Iterables.transform( diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java index 353cf9d0ab90..01656810a33b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java @@ -39,10 +39,10 @@ import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; import org.apache.spark.api.java.function.PairFlatMapFunction; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/ReifyTimestampsAndWindowsFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/ReifyTimestampsAndWindowsFunction.java index dfb58c5c4c53..37f9d1b4e86b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/ReifyTimestampsAndWindowsFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/ReifyTimestampsAndWindowsFunction.java @@ -17,9 +17,10 @@ */ package org.apache.beam.runners.spark.translation; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.spark.api.java.function.Function; /** @@ -32,7 +33,7 @@ public class ReifyTimestampsAndWindowsFunction public KV> call(WindowedValue> elem) throws Exception { return KV.of( elem.getValue().getKey(), - WindowedValue.of( + WindowedValues.of( elem.getValue().getValue(), elem.getTimestamp(), elem.getWindows(), elem.getPane())); } } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SideInputMetadata.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SideInputMetadata.java index 6c407806812c..3e7cd6e700f1 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SideInputMetadata.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SideInputMetadata.java @@ -20,7 +20,7 @@ import java.io.Serializable; import org.apache.beam.runners.spark.util.SideInputBroadcast; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** * Metadata class for side inputs in Spark runner. Contains serialized data, type information and diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAssignWindowFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAssignWindowFn.java index fd00cadf3d40..8c0e9a341266 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAssignWindowFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAssignWindowFn.java @@ -22,7 +22,8 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Window.Assign; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.spark.api.java.function.Function; import org.joda.time.Instant; @@ -62,6 +63,6 @@ public BoundedWindow window() { return boundedWindow; } }); - return WindowedValue.of(element, timestamp, windows, PaneInfo.NO_FIRING); + return WindowedValues.of(element, timestamp, windows, PaneInfo.NO_FIRING); } } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkBatchPortablePipelineTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkBatchPortablePipelineTranslator.java index e37941a35557..ba3aa0e4d24a 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkBatchPortablePipelineTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkBatchPortablePipelineTranslator.java @@ -49,8 +49,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.construction.NativeTransforms; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; @@ -58,6 +56,9 @@ import org.apache.beam.sdk.util.construction.graph.PipelineNode.PTransformNode; import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @@ -174,8 +175,8 @@ private static void translateGroupByKey( Coder inputValueCoder = inputKvCoder.getValueCoder(); WindowingStrategy windowingStrategy = getWindowingStrategy(inputId, components); WindowFn windowFn = windowingStrategy.getWindowFn(); - WindowedValue.WindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of(inputValueCoder, windowFn.windowCoder()); + WindowedValues.WindowedValueCoder wvCoder = + WindowedValues.FullWindowedValueCoder.of(inputValueCoder, windowFn.windowCoder()); JavaRDD>>> groupedByKeyAndWindow; Partitioner partitioner = getPartitioner(context); @@ -233,7 +234,7 @@ private static void translateExecutableStage( Coder> windowedInputCoder = instantiateCoder(inputPCollectionId, components); Coder valueCoder = - ((WindowedValue.FullWindowedValueCoder) windowedInputCoder).getValueCoder(); + ((WindowedValues.FullWindowedValueCoder) windowedInputCoder).getValueCoder(); // Stateful stages are only allowed of KV input to be able to group on the key if (!(valueCoder instanceof KvCoder)) { throw new IllegalStateException( @@ -247,8 +248,8 @@ private static void translateExecutableStage( Coder innerValueCoder = ((KvCoder) valueCoder).getValueCoder(); WindowingStrategy windowingStrategy = getWindowingStrategy(inputPCollectionId, components); WindowFn windowFn = windowingStrategy.getWindowFn(); - WindowedValue.WindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of(innerValueCoder, windowFn.windowCoder()); + WindowedValues.WindowedValueCoder wvCoder = + WindowedValues.FullWindowedValueCoder.of(innerValueCoder, windowFn.windowCoder()); JavaPairRDD>> groupedByKey = groupByKeyPair(inputDataset, keyCoder, wvCoder); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkCombineFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkCombineFn.java index 1075ae0d2a7d..667fa7181bcb 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkCombineFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkCombineFn.java @@ -49,11 +49,12 @@ import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; @@ -244,7 +245,7 @@ public void merge( public Collection> extractOutput() { if (windowAccumulator != null) { return Collections.singletonList( - WindowedValue.of( + WindowedValues.of( windowAccumulator, accTimestamp, accWindow, PaneInfo.ON_TIME_AND_ONLY_FIRING)); } return Collections.emptyList(); @@ -309,7 +310,7 @@ public void add(WindowedValue value, SparkCombineFn context) throws E // merge this.map.put( window, - WindowedValue.of( + WindowedValues.of( context.combineFn.mergeAccumulators( Lists.newArrayList(thisAcc.getValue(), acc.getValue()), context.ctxtForValue(acc)), @@ -442,7 +443,7 @@ void mergeWindows(SparkCombineFn fn) throws Exception { toBeMerged.forEach(this.map::remove); this.map.put( mergeResult.getKey(), - WindowedValue.of( + WindowedValues.of( mergeResult.getValue(), mergedInstant, mergeResult.getKey(), @@ -504,7 +505,7 @@ static class WindowedAccumulatorCoder WindowedAccumulator.Type type) { this.toValue = toValue; - this.accumCoder = WindowedValue.FullWindowedValueCoder.of(accumCoder, windowCoder); + this.accumCoder = WindowedValues.FullWindowedValueCoder.of(accumCoder, windowCoder); this.windowComparator = windowComparator; this.wrap = IterableCoder.of(this.accumCoder); this.type = type; @@ -524,7 +525,7 @@ public void encode(WindowedAccumulator value, OutputS } else { outStream.write(1); accumCoder.encode( - WindowedValue.of( + WindowedValues.of( swwa.windowAccumulator, swwa.accTimestamp, swwa.accWindow, PaneInfo.NO_FIRING), outStream); } @@ -809,9 +810,6 @@ private WindowedAccumulator.Type getType(WindowingStrategy windowingStrate } private static BoundedWindow getWindow(WindowedValue value) { - if (value.isSingleWindowedValue()) { - return ((WindowedValue.SingleWindowedValue) value).getWindow(); - } return Iterables.getOnlyElement(value.getWindows()); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageExtractionFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageExtractionFunction.java index be76e5c0ae08..ae38d097e69d 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageExtractionFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageExtractionFunction.java @@ -20,7 +20,7 @@ import java.util.Collections; import java.util.Iterator; import org.apache.beam.sdk.transforms.join.RawUnionValue; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.spark.api.java.function.FlatMapFunction; @SuppressWarnings({ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java index 4863f7f1f163..757740e2df5a 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java @@ -59,10 +59,10 @@ import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.broadcast.Broadcast; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowViaOutputBufferFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowViaOutputBufferFn.java index cff2cf6bc684..ea620bfe6605 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowViaOutputBufferFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowViaOutputBufferFn.java @@ -35,10 +35,11 @@ import org.apache.beam.runners.core.triggers.TriggerStateMachines; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.spark.api.java.function.FlatMapFunction; import org.joda.time.Instant; @@ -146,7 +147,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - outputs.add(WindowedValue.of(output, timestamp, windows, pane)); + outputs.add(WindowedValues.of(output, timestamp, windows, pane)); } @Override diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessor.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessor.java index 4b4d23b0c47c..d202ac04cac0 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessor.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessor.java @@ -34,8 +34,8 @@ import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.AbstractIterator; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPCollectionView.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPCollectionView.java index 4cc25bd6ffa2..6802cb5aa8e7 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPCollectionView.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPCollectionView.java @@ -23,8 +23,8 @@ import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.runners.spark.util.SideInputBroadcast; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.spark.api.java.JavaSparkContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkStreamingPortablePipelineTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkStreamingPortablePipelineTranslator.java index a252cc044fcb..4850f886241b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkStreamingPortablePipelineTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkStreamingPortablePipelineTranslator.java @@ -47,13 +47,14 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.sdk.util.construction.graph.PipelineNode; import org.apache.beam.sdk.util.construction.graph.PipelineNode.PTransformNode; import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @@ -142,14 +143,14 @@ private static void translateImpulse( Iterable> windowedValues = Collections.singletonList( - WindowedValue.of( + WindowedValues.of( new byte[0], BoundedWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); - WindowedValue.FullWindowedValueCoder windowCoder = - WindowedValue.FullWindowedValueCoder.of(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE); + WindowedValues.FullWindowedValueCoder windowCoder = + WindowedValues.FullWindowedValueCoder.of(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE); JavaRDD> emptyByteArrayRDD = context .getSparkContext() @@ -184,13 +185,13 @@ private static void translateGroupByKey( UnboundedDataset> inputDataset = (UnboundedDataset>) context.popDataset(inputId); List streamSources = inputDataset.getStreamSources(); - WindowedValue.WindowedValueCoder> inputCoder = + WindowedValues.WindowedValueCoder> inputCoder = getWindowedValueCoder(inputId, components); KvCoder inputKvCoder = (KvCoder) inputCoder.getValueCoder(); WindowingStrategy windowingStrategy = getWindowingStrategy(inputId, components); WindowFn windowFn = windowingStrategy.getWindowFn(); - WindowedValue.WindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of( + WindowedValues.WindowedValueCoder wvCoder = + WindowedValues.FullWindowedValueCoder.of( inputKvCoder.getValueCoder(), windowFn.windowCoder()); JavaDStream>>> outStream = @@ -238,7 +239,7 @@ private static void translateExecutableStage( "Side inputs to executable stage are currently unsupported."); } ImmutableMap< - String, Tuple2>, WindowedValue.WindowedValueCoder>> + String, Tuple2>, WindowedValues.WindowedValueCoder>> broadcastVariables = ImmutableMap.copyOf(new HashMap<>()); SparkExecutableStageFunction function = @@ -344,7 +345,7 @@ private static void translateReshuffle( UnboundedDataset inputDataset = (UnboundedDataset) context.popDataset(inputId); List streamSources = inputDataset.getStreamSources(); JavaDStream> dStream = inputDataset.getDStream(); - WindowedValue.WindowedValueCoder coder = + WindowedValues.WindowedValueCoder coder = getWindowedValueCoder(inputId, pipeline.getComponents()); JavaDStream> reshuffledStream = diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java index 1fea8b9329c6..12448b19e1bb 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java @@ -61,7 +61,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.CombineFnUtil; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformMatchers; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.ParDoTranslation; @@ -72,6 +71,8 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.AbstractIterator; @@ -144,8 +145,8 @@ public void evaluate(GroupByKey transform, EvaluationContext context) { // --- coders. final Coder keyCoder = coder.getKeyCoder(); - final WindowedValue.WindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of(coder.getValueCoder(), windowFn.windowCoder()); + final WindowedValues.WindowedValueCoder wvCoder = + WindowedValues.FullWindowedValueCoder.of(coder.getValueCoder(), windowFn.windowCoder()); JavaRDD>>> groupedByKey; Partitioner partitioner = getPartitioner(context); @@ -224,7 +225,7 @@ public void evaluate( JavaRDD>> outRDD = inRDD.map( in -> - WindowedValue.of( + WindowedValues.of( KV.of( in.getValue().getKey(), combineFn.apply( @@ -256,8 +257,8 @@ public void evaluate(Combine.Globally transform, EvaluationCont final CombineWithContext.CombineFnWithContext combineFn = (CombineWithContext.CombineFnWithContext) CombineFnUtil.toFnWithContext(transform.getFn()); - final WindowedValue.FullWindowedValueCoder wvoCoder = - WindowedValue.FullWindowedValueCoder.of( + final WindowedValues.FullWindowedValueCoder wvoCoder = + WindowedValues.FullWindowedValueCoder.of( oCoder, windowingStrategy.getWindowFn().windowCoder()); final boolean hasDefault = transform.isInsertDefault(); @@ -299,7 +300,7 @@ public void evaluate(Combine.Globally transform, EvaluationCont outRdd = jsc.parallelize(Lists.newArrayList(CoderHelpers.toByteArray(defaultValue, oCoder))) .map(CoderHelpers.fromByteFunction(oCoder)) - .map(WindowedValue::valueInGlobalWindow); + .map(WindowedValues::valueInGlobalWindow); } else { outRdd = jsc.emptyRDD(); } @@ -557,8 +558,8 @@ private static JavaPairRDD, WindowedValue> statef boolean requiresSortedInput) { Coder keyCoder = kvCoder.getKeyCoder(); - final WindowedValue.WindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of(kvCoder.getValueCoder(), windowCoder); + final WindowedValues.WindowedValueCoder wvCoder = + WindowedValues.FullWindowedValueCoder.of(kvCoder.getValueCoder(), windowCoder); if (!requiresSortedInput) { return GroupCombineFunctions.groupByKeyOnly(kvInRDD, keyCoder, wvCoder, partitioner) @@ -697,7 +698,7 @@ protected WindowedValue> computeNext() { if (Arrays.equals(prefix, keyPart)) { WindowedValue wv = CoderHelpers.fromByteArray(read._2(), wvCoder); consumed(); - return WindowedValue.of( + return WindowedValues.of( KV.of(key, wv.getValue()), wv.getTimestamp(), wv.getWindows(), wv.getPane()); } } @@ -786,7 +787,7 @@ public void evaluate( Coder>> coderInternal = (Coder) IterableCoder.of( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( output.getCoderInternal(), output.getWindowingStrategyInternal().getWindowFn().windowCoder())); @@ -817,8 +818,8 @@ public void evaluate(Reshuffle transform, EvaluationContext context) { @SuppressWarnings("unchecked") final WindowFn windowFn = (WindowFn) windowingStrategy.getWindowFn(); - final WindowedValue.WindowedValueCoder> wvCoder = - WindowedValue.FullWindowedValueCoder.of(coder, windowFn.windowCoder()); + final WindowedValues.WindowedValueCoder> wvCoder = + WindowedValues.FullWindowedValueCoder.of(coder, windowFn.windowCoder()); JavaRDD>> reshuffled = GroupCombineFunctions.reshuffle(inRDD, wvCoder); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java index f2455e64b956..8d66125efbc6 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java @@ -40,11 +40,12 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; @@ -102,7 +103,7 @@ public CombineGroupedValues(SparkCombineFn, InputT, ?, OutputT> fn @Override public WindowedValue> call(WindowedValue>> windowedKv) throws Exception { - return WindowedValue.of( + return WindowedValues.of( KV.of( windowedKv.getValue().getKey(), fn.getCombineFn() @@ -414,7 +415,7 @@ public static Map, Coder>> getTupleTagCoders( pCollection.getWindowingStrategy().getWindowFn().windowCoder(); @SuppressWarnings("unchecked") Coder> windowedValueCoder = - (Coder>) (Coder) WindowedValue.getFullCoder(coder, wCoder); + (Coder>) (Coder) WindowedValues.getFullCoder(coder, wCoder); coderMap.put(output.getKey(), windowedValueCoder); } return coderMap; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/ParDoStateUpdateFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/ParDoStateUpdateFn.java index b9c7e7d6d63d..1be4042a151d 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/ParDoStateUpdateFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/ParDoStateUpdateFn.java @@ -47,10 +47,11 @@ import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.spark.streaming.State; @@ -93,7 +94,7 @@ public class ParDoStateUpdateFn, O private final String stepName; private final DoFn doFn; private final Coder keyCoder; - private final WindowedValue.FullWindowedValueCoder wvCoder; + private final WindowedValues.FullWindowedValueCoder wvCoder; private transient boolean wasSetupCalled; private final SerializablePipelineOptions options; private final TupleTag mainOutputTag; @@ -116,7 +117,7 @@ public ParDoStateUpdateFn( String stepName, DoFn doFn, Coder keyCoder, - WindowedValue.FullWindowedValueCoder wvCoder, + WindowedValues.FullWindowedValueCoder wvCoder, SerializablePipelineOptions options, TupleTag mainOutputTag, List> additionalOutputTags, @@ -257,8 +258,8 @@ public TimerInternals timerInternals() { (Coder) outputCoders.get(tupleTag); @SuppressWarnings("nullness") - final WindowedValue.FullWindowedValueCoder outputWindowCoder = - WindowedValue.FullWindowedValueCoder.of(outputCoder, windowCoder); + final WindowedValues.FullWindowedValueCoder outputWindowCoder = + WindowedValues.FullWindowedValueCoder.of(outputCoder, windowCoder); return Tuple2.apply( tupleTag, diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StatefulStreamingParDoEvaluator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StatefulStreamingParDoEvaluator.java index c135d6edbf8a..fdd5c180c21e 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StatefulStreamingParDoEvaluator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StatefulStreamingParDoEvaluator.java @@ -45,12 +45,13 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; @@ -126,8 +127,8 @@ public void evaluate( final Coder keyCoder = inputCoder.getKeyCoder(); final Coder valueCoder = inputCoder.getValueCoder(); - final WindowedValue.FullWindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of(valueCoder, windowFn.windowCoder()); + final WindowedValues.FullWindowedValueCoder wvCoder = + WindowedValues.FullWindowedValueCoder.of(valueCoder, windowFn.windowCoder()); final MetricsContainerStepMapAccumulator metricsAccum = MetricsAccumulator.getInstance(); final Map, KV, SideInputBroadcast>> sideInputs = @@ -173,7 +174,7 @@ public void evaluate( stepName, doFn, keyCoder, - (WindowedValue.FullWindowedValueCoder) wvCoder, + (WindowedValues.FullWindowedValueCoder) wvCoder, options, transform.getMainOutputTag(), transform.getAdditionalOutputTags().getAll(), @@ -198,7 +199,7 @@ public void evaluate( final WindowedValue windowedValue = CoderHelpers.fromByteArray( tuple._2(), - WindowedValue.FullWindowedValueCoder.of( + WindowedValues.FullWindowedValueCoder.of( outputCoder, windowFn.windowCoder())); return Tuple2.apply(tuple._1(), windowedValue); })); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java index b505d9bb3d41..0963a3c7a750 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java @@ -76,7 +76,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.CombineFnUtil; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.util.construction.SplittableParDo; @@ -87,6 +86,8 @@ import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -140,7 +141,7 @@ public void evaluate(Impulse transform, EvaluationContext context) { context .getSparkContext() .parallelize( - Collections.singletonList(WindowedValue.valueInGlobalWindow(new byte[0]))); + Collections.singletonList(WindowedValues.valueInGlobalWindow(new byte[0]))); ConstantInputDStream> inputStream = new ConstantInputDStream<>(context.getStreamingContext().ssc(), rdd.rdd(), classTag); JavaDStream> stream = new JavaDStream<>(inputStream, classTag); @@ -227,8 +228,8 @@ public void evaluate(CreateStream transform, EvaluationContext context) { private Queue>> buildRdds( Queue>> batches, JavaStreamingContext jssc, Coder coder) { - final WindowedValue.FullWindowedValueCoder windowCoder = - WindowedValue.FullWindowedValueCoder.of(coder, GlobalWindow.Coder.INSTANCE); + final WindowedValues.FullWindowedValueCoder windowCoder = + WindowedValues.FullWindowedValueCoder.of(coder, GlobalWindow.Coder.INSTANCE); final Queue>> rddQueue = new LinkedBlockingQueue<>(); @@ -237,7 +238,7 @@ private Queue>> buildRdds( StreamSupport.stream(timestampedValues.spliterator(), false) .map( timestampedValue -> - WindowedValue.of( + WindowedValues.of( timestampedValue.getValue(), timestampedValue.getTimestamp(), GlobalWindow.INSTANCE, @@ -368,8 +369,8 @@ public void evaluate(GroupByKey transform, EvaluationContext context) { final WindowFn windowFn = (WindowFn) windowingStrategy.getWindowFn(); // --- coders. - final WindowedValue.WindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of(coder.getValueCoder(), windowFn.windowCoder()); + final WindowedValues.WindowedValueCoder wvCoder = + WindowedValues.FullWindowedValueCoder.of(coder.getValueCoder(), windowFn.windowCoder()); JavaDStream>>> outStream = SparkGroupAlsoByWindowViaWindowSet.groupByKeyAndWindow( @@ -410,7 +411,7 @@ public void evaluate( Coder> coderInternal = (Coder) - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( ListCoder.of(output.getCoderInternal()), output.getWindowingStrategyInternal().getWindowFn().windowCoder()); @@ -634,8 +635,8 @@ public void evaluate(Reshuffle transform, EvaluationContext context) { @SuppressWarnings("unchecked") final WindowFn windowFn = (WindowFn) windowingStrategy.getWindowFn(); - final WindowedValue.WindowedValueCoder> wvCoder = - WindowedValue.FullWindowedValueCoder.of(coder, windowFn.windowCoder()); + final WindowedValues.WindowedValueCoder> wvCoder = + WindowedValues.FullWindowedValueCoder.of(coder, windowFn.windowCoder()); JavaDStream>> reshuffledStream = dStream.transform(rdd -> GroupCombineFunctions.reshuffle(rdd, wvCoder)); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/TestDStream.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/TestDStream.java index e0bda8ba9491..abf2e7c3242e 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/TestDStream.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/TestDStream.java @@ -32,8 +32,9 @@ import org.apache.beam.sdk.testing.TestStream.WatermarkEvent; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.Preconditions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext$; @@ -61,7 +62,7 @@ public class TestDStream extends InputDStream> { public TestDStream(TestStream test, StreamingContext ssc) { super(ssc, classTag()); - this.coder = WindowedValue.getFullCoder(test.getValueCoder(), GlobalWindow.Coder.INSTANCE); + this.coder = WindowedValues.getFullCoder(test.getValueCoder(), GlobalWindow.Coder.INSTANCE); this.events = test.getEvents(); } @@ -126,7 +127,7 @@ private RDD> buildRdd(ElementEvent event) { List binaryData = new ArrayList<>(); for (TimestampedValue elem : event.getElements()) { WindowedValue wv = - WindowedValue.timestampedValueInGlobalWindow(elem.getValue(), elem.getTimestamp()); + WindowedValues.timestampedValueInGlobalWindow(elem.getValue(), elem.getTimestamp()); binaryData.add(CoderHelpers.toByteArray(wv, coder)); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/UnboundedDataset.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/UnboundedDataset.java index ec7e00d24fcf..3c4674cd2260 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/UnboundedDataset.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/UnboundedDataset.java @@ -23,7 +23,7 @@ import org.apache.beam.runners.spark.translation.Dataset; import org.apache.beam.runners.spark.translation.TranslationUtils; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.api.java.JavaDStream; import org.slf4j.Logger; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/WatermarkSyncedDStream.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/WatermarkSyncedDStream.java index fef2d2ec675e..d9c40be501c0 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/WatermarkSyncedDStream.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/WatermarkSyncedDStream.java @@ -22,7 +22,7 @@ import java.util.Queue; import java.util.concurrent.TimeUnit; import org.apache.beam.runners.spark.util.GlobalWatermarkHolder; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Stopwatch; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles; import org.apache.spark.api.java.JavaRDD; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java index 4e24d7e50e6a..414f2abc01a9 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java @@ -34,10 +34,10 @@ import org.apache.beam.sdk.transforms.Materializations.MultimapView; import org.apache.beam.sdk.transforms.ViewFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctionsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctionsTest.java index fd299924af91..f4d154fa772f 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctionsTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctionsTest.java @@ -34,9 +34,10 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.primitives.Bytes; import org.joda.time.Duration; @@ -123,8 +124,8 @@ private GroupByKeyIterator createG throws Coder.NonDeterministicException { StringUtf8Coder keyCoder = StringUtf8Coder.of(); - final WindowedValue.FullWindowedValueCoder> winValCoder = - WindowedValue.getFullCoder( + final WindowedValues.FullWindowedValueCoder> winValCoder = + WindowedValues.getFullCoder( KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()), winStrategy.getWindowFn().windowCoder()); @@ -157,7 +158,7 @@ static ItemFactory forWindow( } private final Coder keyCoder; - private final WindowedValue.FullWindowedValueCoder> winValCoder; + private final WindowedValues.FullWindowedValueCoder> winValCoder; private final byte[] windowBytes; private final W window; @@ -178,7 +179,7 @@ private Tuple2 create(K key, V value) { byte[] windowedValue = CoderHelpers.toByteArray( - WindowedValue.of( + WindowedValues.of( KV.of(key, value), Instant.now(), window, PaneInfo.ON_TIME_AND_ONLY_FIRING), winValCoder); return new Tuple2<>(kaw, windowedValue); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkCombineFnTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkCombineFnTest.java index 9cb4b44c897c..f56a20b49151 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkCombineFnTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkCombineFnTest.java @@ -39,8 +39,9 @@ import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.CombineFnUtil; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -233,7 +234,7 @@ public void testGlobalWindowMergeAccumulatorsWithEarliestCombiner() throws Excep WindowedValue> first = input("key", 1, ts); WindowedValue> second = input("key", 2, ts); WindowedValue> third = input("key", 3, ts); - WindowedValue accumulator = WindowedValue.valueInGlobalWindow(0L); + WindowedValue accumulator = WindowedValues.valueInGlobalWindow(0L); SparkCombineFn.SingleWindowWindowedAccumulator, Integer, Long> acc1 = SparkCombineFn.SingleWindowWindowedAccumulator.create(KV::getValue, accumulator); SparkCombineFn.SingleWindowWindowedAccumulator, Integer, Long> acc2 = @@ -293,7 +294,7 @@ private WindowedValue inputValue(V value, Instant timestamp, WindowFn cast = (WindowFn) windowFn; - return WindowedValue.of( + return WindowedValues.of( value, timestamp, cast.assignWindows(assignContext(cast, value, timestamp)), diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunctionTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunctionTest.java index 7276820e99e8..98601389f5c9 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunctionTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunctionTest.java @@ -55,9 +55,10 @@ import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.join.RawUnionValue; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.Before; import org.junit.Test; @@ -116,7 +117,7 @@ public void sdkErrorsSurfaceOnClose() throws Exception { SparkExecutableStageFunction function = getFunction(Collections.emptyMap()); doThrow(new Exception()).when(remoteBundle).close(); List> inputs = new ArrayList<>(); - inputs.add(WindowedValue.valueInGlobalWindow(0)); + inputs.add(WindowedValues.valueInGlobalWindow(0)); function.call(inputs.iterator()); } @@ -132,9 +133,9 @@ public void expectedInputsAreSent() throws Exception { FnDataReceiver> receiver = Mockito.mock(FnDataReceiver.class); when(bundle.getInputReceivers()).thenReturn(ImmutableMap.of(inputId, receiver)); - WindowedValue one = WindowedValue.valueInGlobalWindow(1); - WindowedValue two = WindowedValue.valueInGlobalWindow(2); - WindowedValue three = WindowedValue.valueInGlobalWindow(3); + WindowedValue one = WindowedValues.valueInGlobalWindow(1); + WindowedValue two = WindowedValues.valueInGlobalWindow(2); + WindowedValue three = WindowedValues.valueInGlobalWindow(3); function.call(Arrays.asList(one, two, three).iterator()); verify(receiver).accept(one); @@ -145,9 +146,9 @@ public void expectedInputsAreSent() throws Exception { @Test public void outputsAreTaggedCorrectly() throws Exception { - WindowedValue three = WindowedValue.valueInGlobalWindow(3); - WindowedValue four = WindowedValue.valueInGlobalWindow(4); - WindowedValue five = WindowedValue.valueInGlobalWindow(5); + WindowedValue three = WindowedValues.valueInGlobalWindow(3); + WindowedValue four = WindowedValues.valueInGlobalWindow(4); + WindowedValue five = WindowedValues.valueInGlobalWindow(5); Map outputTagMap = ImmutableMap.of( "one", 1, @@ -230,7 +231,7 @@ public void close() {} SparkExecutableStageFunction function = getFunction(outputTagMap); List> inputs = new ArrayList<>(); - inputs.add(WindowedValue.valueInGlobalWindow(0)); + inputs.add(WindowedValues.valueInGlobalWindow(0)); Iterator iterator = function.call(inputs.iterator()); Iterable iterable = () -> iterator; @@ -244,7 +245,7 @@ public void close() {} public void testStageBundleClosed() throws Exception { SparkExecutableStageFunction function = getFunction(Collections.emptyMap()); List> inputs = new ArrayList<>(); - inputs.add(WindowedValue.valueInGlobalWindow(0)); + inputs.add(WindowedValues.valueInGlobalWindow(0)); function.call(inputs.iterator()); verify(stageBundleFactory).getBundle(any(), any(), any(), any(BundleProgressHandler.class)); verify(stageBundleFactory).getProcessBundleDescriptor(); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java index be2b788f12f5..c6ccec7bf451 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java @@ -35,8 +35,9 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.joda.time.Instant; @@ -101,7 +102,7 @@ public void testBoundedProcessBlocksOnMaxInputsUntilTheyAreConsumed() throws Exc setUpCtx(processor.getOutputManager(), desiredWriteCount, writeCount); Iterator> input = - Lists.newArrayList(WindowedValue.valueInGlobalWindow("tick")).iterator(); + Lists.newArrayList(WindowedValues.valueInGlobalWindow("tick")).iterator(); Iterator result = processor.createOutputIterator(input, ctx); CountDownLatch maxReached = new CountDownLatch(1); @@ -146,7 +147,7 @@ public void testUnboundedProcessWritesAllInputsAndNotBlock() { setUpCtx(processor.getOutputManager(), desiredWriteCount, writeCount); Iterator> input = - Lists.newArrayList(WindowedValue.valueInGlobalWindow("tick")).iterator(); + Lists.newArrayList(WindowedValues.valueInGlobalWindow("tick")).iterator(); Iterator result = processor.createOutputIterator(input, ctx); // this will trigger input processing via doFn @@ -168,7 +169,7 @@ public void testBoundedProcessLifecycle() { SparkProcessContext ctx = setUpCtx(processor.getOutputManager(), desiredWriteCount, writeCount); - WindowedValue value = WindowedValue.valueInGlobalWindow("tick"); + WindowedValue value = WindowedValues.valueInGlobalWindow("tick"); Iterator> input = Lists.newArrayList(value).iterator(); Iterator result = processor.createOutputIterator(input, ctx); @@ -193,7 +194,7 @@ public void testUnboundedProcessLifecycle() { SparkProcessContext ctx = setUpCtx(processor.getOutputManager(), desiredWriteCount, writeCount); - WindowedValue value = WindowedValue.valueInGlobalWindow("tick"); + WindowedValue value = WindowedValues.valueInGlobalWindow("tick"); Iterator> input = Lists.newArrayList(value).iterator(); Iterator result = processor.createOutputIterator(input, ctx); @@ -269,7 +270,7 @@ class TestDoFn extends DoFn { @DoFn.ProcessElement public void processElement(@Element String value) { for (int i = 0; i < desiredCount; i++) { - output.output(new TupleTag<>("key"), WindowedValue.valueInGlobalWindow(value + "_" + i)); + output.output(new TupleTag<>("key"), WindowedValues.valueInGlobalWindow(value + "_" + i)); producedCount.incrementAndGet(); } } diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java index 2faf38a106df..f5ad228152fc 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java @@ -43,12 +43,13 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -81,8 +82,8 @@ public void testIteratorFlatten() { @Test public void testSplitBySameKey() { VarIntCoder coder = VarIntCoder.of(); - WindowedValue.WindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of(coder, GlobalWindow.Coder.INSTANCE); + WindowedValues.WindowedValueCoder wvCoder = + WindowedValues.FullWindowedValueCoder.of(coder, GlobalWindow.Coder.INSTANCE); Instant now = Instant.now(); List window = Collections.singletonList(GlobalWindow.INSTANCE); PaneInfo paneInfo = PaneInfo.NO_FIRING; @@ -90,23 +91,25 @@ public void testSplitBySameKey() { Arrays.asList( new Tuple2( new ByteArray(CoderHelpers.toByteArrayWithTs(1, coder, now)), - CoderHelpers.toByteArray(WindowedValue.of(1, now, window, paneInfo), wvCoder)), + CoderHelpers.toByteArray(WindowedValues.of(1, now, window, paneInfo), wvCoder)), new Tuple2( new ByteArray( CoderHelpers.toByteArrayWithTs(1, coder, now.plus(Duration.millis(1)))), CoderHelpers.toByteArray( - WindowedValue.of(2, now.plus(Duration.millis(1)), window, paneInfo), wvCoder))); + WindowedValues.of(2, now.plus(Duration.millis(1)), window, paneInfo), + wvCoder))); List> secondKey = Arrays.asList( new Tuple2( new ByteArray(CoderHelpers.toByteArrayWithTs(2, coder, now)), - CoderHelpers.toByteArray(WindowedValue.of(3, now, window, paneInfo), wvCoder)), + CoderHelpers.toByteArray(WindowedValues.of(3, now, window, paneInfo), wvCoder)), new Tuple2( new ByteArray( CoderHelpers.toByteArrayWithTs(2, coder, now.plus(Duration.millis(2)))), CoderHelpers.toByteArray( - WindowedValue.of(4, now.plus(Duration.millis(2)), window, paneInfo), wvCoder))); + WindowedValues.of(4, now.plus(Duration.millis(2)), window, paneInfo), + wvCoder))); Iterable> concat = Iterables.concat(firstKey, secondKey); Iterator>>> keySplit; @@ -120,15 +123,15 @@ public void testSplitBySameKey() { // first key assertEquals( Arrays.asList( - WindowedValue.of(KV.of(1, 1), now, window, paneInfo), - WindowedValue.of(KV.of(1, 2), now.plus(Duration.millis(1)), window, paneInfo)), + WindowedValues.of(KV.of(1, 1), now, window, paneInfo), + WindowedValues.of(KV.of(1, 2), now.plus(Duration.millis(1)), window, paneInfo)), list); } else { // second key assertEquals( Arrays.asList( - WindowedValue.of(KV.of(2, 3), now, window, paneInfo), - WindowedValue.of(KV.of(2, 4), now.plus(Duration.millis(2)), window, paneInfo)), + WindowedValues.of(KV.of(2, 3), now, window, paneInfo), + WindowedValues.of(KV.of(2, 4), now.plus(Duration.millis(2)), window, paneInfo)), list); } } diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/ValueAndCoderLazySerializableTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/ValueAndCoderLazySerializableTest.java index 450d3e0bb013..0f2ef2ce5f45 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/ValueAndCoderLazySerializableTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/ValueAndCoderLazySerializableTest.java @@ -32,7 +32,8 @@ import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.joda.time.Instant; import org.junit.Test; @@ -48,8 +49,8 @@ public void serializableAccumulatorSerializationTest() Iterable> accumulatedValue = Arrays.asList(winVal(0), winVal(1), winVal(3), winVal(4)); - final WindowedValue.FullWindowedValueCoder wvaCoder = - WindowedValue.FullWindowedValueCoder.of( + final WindowedValues.FullWindowedValueCoder wvaCoder = + WindowedValues.FullWindowedValueCoder.of( BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE); final IterableCoder> iterAccumCoder = IterableCoder.of(wvaCoder); @@ -74,8 +75,8 @@ public void serializableAccumulatorKryoTest() { Iterable> accumulatedValue = Arrays.asList(winVal(0), winVal(1), winVal(3), winVal(4)); - final WindowedValue.FullWindowedValueCoder wvaCoder = - WindowedValue.FullWindowedValueCoder.of( + final WindowedValues.FullWindowedValueCoder wvaCoder = + WindowedValues.FullWindowedValueCoder.of( BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE); final IterableCoder> iterAccumCoder = IterableCoder.of(wvaCoder); @@ -104,6 +105,6 @@ public void serializableAccumulatorKryoTest() { } private WindowedValue winVal(T val) { - return WindowedValue.of(val, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); + return WindowedValues.of(val, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); } } diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2BatchTranslationContext.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2BatchTranslationContext.java index 0acfe6af5f79..628425a52e45 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2BatchTranslationContext.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2BatchTranslationContext.java @@ -20,8 +20,8 @@ import edu.iu.dsc.tws.tset.env.BatchTSetEnvironment; import edu.iu.dsc.tws.tset.sets.batch.BatchTSetImpl; import edu.iu.dsc.tws.tset.sets.batch.SinkTSet; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.WindowedValue; /** Twister2BatchTranslationContext. */ public class Twister2BatchTranslationContext extends Twister2TranslationContext { diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2TranslationContext.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2TranslationContext.java index 7102e8f8ef0e..a194d86f7460 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2TranslationContext.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2TranslationContext.java @@ -29,11 +29,11 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; /** Twister2TranslationContext. */ diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2BoundedSource.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2BoundedSource.java index 4525a67b28ea..1bd866d07130 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2BoundedSource.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2BoundedSource.java @@ -34,10 +34,11 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PortablePipelineOptions; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.sdk.util.construction.ReadTranslation; import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; /** Twister2 wrapper for Bounded Source. */ @SuppressWarnings({ @@ -177,7 +178,7 @@ private boolean tryProduceNext() { try { if (seekNext()) { next = - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( reader.getCurrent(), reader.getCurrentTimestamp()); return SUCCESSFULLY_OBTAINED_NEXT; } else { diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2EmptySource.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2EmptySource.java index c15065024fc7..8e875e132a1b 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2EmptySource.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2EmptySource.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.twister2.translation.wrappers; import edu.iu.dsc.tws.api.tset.fn.BaseSourceFunc; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** * Empty Source wrapper. diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/AssignWindowTranslatorBatch.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/AssignWindowTranslatorBatch.java index dbb5f47db4b2..39cba2ed2ddb 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/AssignWindowTranslatorBatch.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/AssignWindowTranslatorBatch.java @@ -26,7 +26,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; /** Assign Window translator. */ diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/FlattenTranslatorBatch.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/FlattenTranslatorBatch.java index 00729e8dbc15..4cf5d347220b 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/FlattenTranslatorBatch.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/FlattenTranslatorBatch.java @@ -28,9 +28,9 @@ import org.apache.beam.runners.twister2.translation.wrappers.Twister2EmptySource; import org.apache.beam.runners.twister2.translators.BatchTransformTranslator; import org.apache.beam.sdk.transforms.Flatten; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.WindowedValue; /** Flatten translator. */ @SuppressWarnings({ diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/GroupByKeyTranslatorBatch.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/GroupByKeyTranslatorBatch.java index a1ec33785a69..78c90b966ac7 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/GroupByKeyTranslatorBatch.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/GroupByKeyTranslatorBatch.java @@ -33,9 +33,10 @@ import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; /** GroupByKey translator. */ @@ -53,8 +54,8 @@ public void translateNode(GroupByKey transform, Twister2BatchTranslationCo WindowingStrategy windowingStrategy = input.getWindowingStrategy(); WindowFn, BoundedWindow> windowFn = (WindowFn, BoundedWindow>) windowingStrategy.getWindowFn(); - final WindowedValue.WindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of(coder.getValueCoder(), windowFn.windowCoder()); + final WindowedValues.WindowedValueCoder wvCoder = + WindowedValues.FullWindowedValueCoder.of(coder.getValueCoder(), windowFn.windowCoder()); KeyedTSet keyedTSet = inputTTSet.mapToTuple(new MapToTupleFunction(inputKeyCoder, wvCoder)); diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ImpulseTranslatorBatch.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ImpulseTranslatorBatch.java index fa876c745027..afc325e06fc4 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ImpulseTranslatorBatch.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ImpulseTranslatorBatch.java @@ -24,8 +24,8 @@ import org.apache.beam.runners.twister2.translators.BatchTransformTranslator; import org.apache.beam.runners.twister2.translators.functions.ImpulseSource; import org.apache.beam.sdk.transforms.Impulse; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; /** Impulse translator. */ public class ImpulseTranslatorBatch implements BatchTransformTranslator { diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/PCollectionViewTranslatorBatch.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/PCollectionViewTranslatorBatch.java index 3b17bdac9e91..d6c69c8a6324 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/PCollectionViewTranslatorBatch.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/PCollectionViewTranslatorBatch.java @@ -32,9 +32,10 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CreatePCollectionViewTranslation; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; /** PCollectionView translator. */ @@ -74,8 +75,8 @@ public void translateNode( case Materializations.MULTIMAP_MATERIALIZATION_URN: KvCoder kvCoder = (KvCoder) coder; final Coder keyCoder = kvCoder.getKeyCoder(); - final WindowedValue.WindowedValueCoder kvwvCoder = - WindowedValue.FullWindowedValueCoder.of( + final WindowedValues.WindowedValueCoder kvwvCoder = + WindowedValues.FullWindowedValueCoder.of( kvCoder.getValueCoder(), windowFn.windowCoder()); BatchTSet> multimapMaterialization = inputDataSet @@ -86,8 +87,8 @@ public void translateNode( context.setSideInputDataSet(input.getTagInternal().getId(), multimapMaterialization); break; case Materializations.ITERABLE_MATERIALIZATION_URN: - final WindowedValue.WindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of(coder, windowFn.windowCoder()); + final WindowedValues.WindowedValueCoder wvCoder = + WindowedValues.FullWindowedValueCoder.of(coder, windowFn.windowCoder()); BatchTSet> iterableMaterialization = inputDataSet .direct() diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ParDoMultiOutputTranslatorBatch.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ParDoMultiOutputTranslatorBatch.java index 97aeff608d7a..9230c6efd868 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ParDoMultiOutputTranslatorBatch.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ParDoMultiOutputTranslatorBatch.java @@ -35,11 +35,11 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ReadSourceTranslatorBatch.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ReadSourceTranslatorBatch.java index 14e858fe5108..ac071db64815 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ReadSourceTranslatorBatch.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ReadSourceTranslatorBatch.java @@ -24,9 +24,9 @@ import org.apache.beam.runners.twister2.translation.wrappers.Twister2BoundedSource; import org.apache.beam.runners.twister2.translators.BatchTransformTranslator; import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; /** Source translator. */ public class ReadSourceTranslatorBatch diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java index cc7b9d9a5373..ad235a72d2e5 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java @@ -30,10 +30,11 @@ import org.apache.beam.sdk.options.PortablePipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; /** Assign Windows function. */ @@ -69,7 +70,7 @@ public void compute(Iterator> input, RecordCollector> map(Tuple input) { WindowedValue> element; if (value == null) { - value = WindowedValue.valueInGlobalWindow(null); + value = WindowedValues.valueInGlobalWindow(null); } element = - WindowedValue.of( + WindowedValues.of( KV.of(key, value.getValue()), value.getTimestamp(), value.getWindows(), diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/DoFnFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/DoFnFunction.java index d8202882b4b6..8cef3618023e 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/DoFnFunction.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/DoFnFunction.java @@ -50,7 +50,6 @@ import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.util.DoFnWithExecutionInformation; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.util.construction.RehydratedComponents; @@ -58,6 +57,7 @@ import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ElemToBytesFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ElemToBytesFunction.java index e2029910cb8d..0121eb2cfcc0 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ElemToBytesFunction.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ElemToBytesFunction.java @@ -24,7 +24,8 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.checkerframework.checker.nullness.qual.Nullable; /** Map to tuple function. */ @@ -33,7 +34,7 @@ }) public class ElemToBytesFunction implements MapFunc> { - private transient WindowedValue.WindowedValueCoder wvCoder; + private transient WindowedValues.WindowedValueCoder wvCoder; private static final Logger LOG = Logger.getLogger(ElemToBytesFunction.class.getName()); private transient boolean isInitialized = false; @@ -44,7 +45,7 @@ public ElemToBytesFunction() { this.isInitialized = false; } - public ElemToBytesFunction(WindowedValue.WindowedValueCoder wvCoder) { + public ElemToBytesFunction(WindowedValues.WindowedValueCoder wvCoder) { this.wvCoder = wvCoder; wvCoderBytes = SerializableUtils.serializeToByteArray(wvCoder); } @@ -73,7 +74,7 @@ private void initTransient() { return; } wvCoder = - (WindowedValue.WindowedValueCoder) + (WindowedValues.WindowedValueCoder) SerializableUtils.deserializeFromByteArray(wvCoderBytes, "Coder"); this.isInitialized = true; } diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/GroupByWindowFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/GroupByWindowFunction.java index be3d25a8740b..3880b8e907cd 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/GroupByWindowFunction.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/GroupByWindowFunction.java @@ -42,7 +42,6 @@ import org.apache.beam.sdk.options.PortablePipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.apache.beam.sdk.util.construction.SdkComponents; @@ -50,6 +49,8 @@ import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; import org.joda.time.Instant; @@ -182,7 +183,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - outputs.add(WindowedValue.of(output, timestamp, windows, pane)); + outputs.add(WindowedValues.of(output, timestamp, windows, pane)); } @Override diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ImpulseSource.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ImpulseSource.java index 6fbceaa148e4..3d53fe9166c1 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ImpulseSource.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ImpulseSource.java @@ -18,12 +18,13 @@ package org.apache.beam.runners.twister2.translators.functions; import edu.iu.dsc.tws.api.tset.fn.SourceFunc; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; /** A {@link SourceFunc} which executes the impulse transform contract. */ public class ImpulseSource implements SourceFunc> { private static final WindowedValue IMPULSE_VALUE = - WindowedValue.valueInGlobalWindow(new byte[0]); + WindowedValues.valueInGlobalWindow(new byte[0]); private boolean impulseEmitted = false; diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/MapToTupleFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/MapToTupleFunction.java index d96fa224ddac..da1080b697ac 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/MapToTupleFunction.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/MapToTupleFunction.java @@ -26,8 +26,9 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; /** Map to tuple function. */ @SuppressWarnings({ @@ -37,7 +38,7 @@ public class MapToTupleFunction implements MapFunc, WindowedValue>> { private transient Coder keyCoder; - private transient WindowedValue.WindowedValueCoder wvCoder; + private transient WindowedValues.WindowedValueCoder wvCoder; private static final Logger LOG = Logger.getLogger(MapToTupleFunction.class.getName()); private transient boolean isInitialized = false; @@ -49,7 +50,7 @@ public MapToTupleFunction() { this.isInitialized = false; } - public MapToTupleFunction(Coder inputKeyCoder, WindowedValue.WindowedValueCoder wvCoder) { + public MapToTupleFunction(Coder inputKeyCoder, WindowedValues.WindowedValueCoder wvCoder) { this.keyCoder = inputKeyCoder; this.wvCoder = wvCoder; keyCoderBytes = SerializableUtils.serializeToByteArray(keyCoder); @@ -61,10 +62,10 @@ public Tuple map(WindowedValue> input) { Tuple element = null; WindowedValue>> temp = - WindowedValue.of( + WindowedValues.of( KV.of( input.getValue().getKey(), - WindowedValue.of( + WindowedValues.of( input.getValue().getValue(), input.getTimestamp(), input.getWindows(), @@ -98,7 +99,7 @@ private void initTransient() { } keyCoder = (Coder) SerializableUtils.deserializeFromByteArray(keyCoderBytes, "Coder"); wvCoder = - (WindowedValue.WindowedValueCoder) + (WindowedValues.WindowedValueCoder) SerializableUtils.deserializeFromByteArray(wvCoderBytes, "Coder"); this.isInitialized = true; } diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/OutputTagFilter.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/OutputTagFilter.java index 7b553a24e36c..b8d00c3925ed 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/OutputTagFilter.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/OutputTagFilter.java @@ -22,7 +22,7 @@ import edu.iu.dsc.tws.api.tset.fn.RecordCollector; import java.util.Iterator; import org.apache.beam.sdk.transforms.join.RawUnionValue; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** Output tag filter. */ @SuppressWarnings({ diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/TranslationUtils.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/TranslationUtils.java index 26e4c6b0db44..faf1abc197de 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/TranslationUtils.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/TranslationUtils.java @@ -22,7 +22,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; /** doc. */ @SuppressWarnings({ @@ -59,7 +60,7 @@ public static T fromByteArray(byte[] serialized, Coder coder) { * @return Deserialized object. */ public static WindowedValue fromByteArray( - byte[] serialized, WindowedValue.WindowedValueCoder coder) { + byte[] serialized, WindowedValues.WindowedValueCoder coder) { try { return CoderUtils.decodeFromByteArray(coder, serialized); } catch (CoderException e) { diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2AssignContext.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2AssignContext.java index 491917a8839c..1155dded04d2 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2AssignContext.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2AssignContext.java @@ -19,7 +19,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.joda.time.Instant; diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2SideInputReader.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2SideInputReader.java index a4735080f8f5..e2e2a281a9fc 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2SideInputReader.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2SideInputReader.java @@ -36,9 +36,9 @@ import org.apache.beam.sdk.transforms.Materializations.MultimapView; import org.apache.beam.sdk.transforms.ViewFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java index dc81e23b10ee..2dce2bed7d38 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java @@ -58,7 +58,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; @@ -66,6 +65,8 @@ import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TimestampedValue.TimestampedValueCoder; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -872,7 +873,9 @@ public PCollection expand(PBegin input) { Pipeline.applyTransform( input, Create.of(windowedValues) - .withCoder(WindowedValue.getFullCoder(coder, windowCoder))); + .withCoder( + org.apache.beam.sdk.values.WindowedValues.getFullCoder( + coder, windowCoder))); PCollection output = intermediate.apply(ParDo.of(new ConvertWindowedValues<>())); output.setCoder(coder); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslators.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslators.java index f82a9759547a..84a90721a983 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslators.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslators.java @@ -34,8 +34,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.InstanceBuilder; import org.apache.beam.sdk.util.ShardedKey; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -122,28 +122,29 @@ public List> getComponents(FullWindowedValueCoder from) { @Override public FullWindowedValueCoder fromComponents(List> components) { - return WindowedValue.getFullCoder( + return WindowedValues.getFullCoder( components.get(0), (Coder) components.get(1)); } }; } - static CoderTranslator> paramWindowedValue() { - return new CoderTranslator>() { + static CoderTranslator> paramWindowedValue() { + return new CoderTranslator>() { @Override - public List> getComponents(WindowedValue.ParamWindowedValueCoder from) { + public List> getComponents( + WindowedValues.ParamWindowedValueCoder from) { return ImmutableList.of(from.getValueCoder(), from.getWindowCoder()); } @Override - public byte[] getPayload(WindowedValue.ParamWindowedValueCoder from) { - return WindowedValue.ParamWindowedValueCoder.getPayload(from); + public byte[] getPayload(WindowedValues.ParamWindowedValueCoder from) { + return WindowedValues.ParamWindowedValueCoder.getPayload(from); } @Override - public WindowedValue.ParamWindowedValueCoder fromComponents( + public WindowedValues.ParamWindowedValueCoder fromComponents( List> components, byte[] payload, CoderTranslation.TranslationContext context) { - return WindowedValue.ParamWindowedValueCoder.fromComponents(components, payload); + return WindowedValues.ParamWindowedValueCoder.fromComponents(components, payload); } }; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ModelCoderRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ModelCoderRegistrar.java index 7fde7bd17484..5b0d5aedd619 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ModelCoderRegistrar.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ModelCoderRegistrar.java @@ -37,8 +37,7 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder; import org.apache.beam.sdk.util.ShardedKey; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableBiMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @@ -66,9 +65,9 @@ public class ModelCoderRegistrar implements CoderTranslatorRegistrar { .put(Timer.Coder.class, ModelCoders.TIMER_CODER_URN) .put(LengthPrefixCoder.class, ModelCoders.LENGTH_PREFIX_CODER_URN) .put(GlobalWindow.Coder.class, ModelCoders.GLOBAL_WINDOW_CODER_URN) - .put(FullWindowedValueCoder.class, ModelCoders.WINDOWED_VALUE_CODER_URN) + .put(WindowedValues.FullWindowedValueCoder.class, ModelCoders.WINDOWED_VALUE_CODER_URN) .put( - WindowedValue.ParamWindowedValueCoder.class, + WindowedValues.ParamWindowedValueCoder.class, ModelCoders.PARAM_WINDOWED_VALUE_CODER_URN) .put(DoubleCoder.class, ModelCoders.DOUBLE_CODER_URN) .put(RowCoder.class, ModelCoders.ROW_CODER_URN) @@ -90,9 +89,10 @@ public class ModelCoderRegistrar implements CoderTranslatorRegistrar { .put(IterableCoder.class, CoderTranslators.iterable()) .put(Timer.Coder.class, CoderTranslators.timer()) .put(LengthPrefixCoder.class, CoderTranslators.lengthPrefix()) - .put(FullWindowedValueCoder.class, CoderTranslators.fullWindowedValue()) .put( - WindowedValue.ParamWindowedValueCoder.class, + WindowedValues.FullWindowedValueCoder.class, CoderTranslators.fullWindowedValue()) + .put( + WindowedValues.ParamWindowedValueCoder.class, CoderTranslators.paramWindowedValue()) .put(DoubleCoder.class, CoderTranslators.atomic(DoubleCoder.class)) .put(RowCoder.class, CoderTranslators.row()) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/RunnerPCollectionView.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/RunnerPCollectionView.java index 3af8836c4622..8d1d02f4b555 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/RunnerPCollectionView.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/RunnerPCollectionView.java @@ -23,12 +23,12 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.ViewFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.PValueBase; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValue.java new file mode 100644 index 000000000000..2a5236f0147f --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValue.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.values; + +import java.util.Collection; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.joda.time.Instant; + +/** + * A value along with Beam's windowing information and all other metadata. + * + * @param the type of the primary data for the value. + */ +public interface WindowedValue { + /** The primary data for this value. */ + T getValue(); + + /** The timestamp of this value in event time. */ + Instant getTimestamp(); + + /** Returns the windows of this {@code WindowedValue}. */ + Collection getWindows(); + + /** @deprecated Do not use; only here for ease of migration/container update. */ + @Deprecated + PaneInfo getPane(); + + /** The {@link PaneInfo} associated with this WindowedValue. */ + default PaneInfo getPaneInfo() { + return getPane(); + } + + /** + * A representation of each of the actual values represented by this compressed {@link + * WindowedValue}, one per window. + */ + Iterable> explodeWindows(); + + /** + * A {@link WindowedValue} with identical metadata to the current one, but with the provided + * value. + */ + WindowedValue withValue(OtherT value); +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValues.java similarity index 88% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValues.java index d11166001f05..3c044990de37 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValues.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.sdk.values; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; @@ -52,12 +52,14 @@ import org.joda.time.Instant; /** - * An immutable triple of value, timestamp, and windows. + * Implementations of {@link WindowedValue} and static utility methods. * - * @param the type of the value + *

These are primarily intended for internal use by Beam SDK developers and runner developers. + * Backwards incompatible changes will likely occur. */ @Internal -public abstract class WindowedValue { +public class WindowedValues { + private WindowedValues() {} // non-instantiable utility class /** Returns a {@code WindowedValue} with the given value, timestamp, and windows. */ public static WindowedValue of( @@ -143,70 +145,47 @@ public static WindowedValue timestampedValueInGlobalWindow( * Returns a new {@code WindowedValue} that is a copy of this one, but with a different value, * which may have a new type {@code NewT}. */ - public abstract WindowedValue withValue(NewT value); - - /** Returns the value of this {@code WindowedValue}. */ - public abstract T getValue(); - - /** Returns the timestamp of this {@code WindowedValue}. */ - public abstract Instant getTimestamp(); - - /** Returns the windows of this {@code WindowedValue}. */ - public abstract Collection getWindows(); - - /** Returns the pane of this {@code WindowedValue} in its window. */ - public abstract PaneInfo getPane(); - - /** Returns {@code true} if this WindowedValue has exactly one window. */ - public boolean isSingleWindowedValue() { - return false; + public static WindowedValue withValue( + WindowedValue windowedValue, NewT newValue) { + return WindowedValues.of( + newValue, + windowedValue.getTimestamp(), + windowedValue.getWindows(), + windowedValue.getPaneInfo()); } - /** - * Returns a collection of {@link WindowedValue WindowedValues} identical to this one, except each - * is in exactly one of the windows that this {@link WindowedValue} is in. - */ - public Iterable> explodeWindows() { - if (isSingleWindowedValue()) { - return ImmutableList.of(this); + public static boolean equals( + @Nullable WindowedValue left, @Nullable WindowedValue right) { + if (left == null) { + return right == null; } - ImmutableList.Builder> windowedValues = ImmutableList.builder(); - for (BoundedWindow w : getWindows()) { - windowedValues.add(of(getValue(), getTimestamp(), w, getPane())); - } - return windowedValues.build(); - } - @Override - public boolean equals(@Nullable Object other) { - if (!(other instanceof WindowedValue)) { + if (right == null) { return false; - } else { - WindowedValue that = (WindowedValue) other; - - // Compare timestamps first as they are most likely to differ. - // Also compare timestamps according to millis-since-epoch because otherwise expensive - // comparisons are made on their Chronology objects. - return this.getTimestamp().isEqual(that.getTimestamp()) - && Objects.equals(this.getValue(), that.getValue()) - && Objects.equals(this.getWindows(), that.getWindows()) - && Objects.equals(this.getPane(), that.getPane()); } + + // Compare timestamps first as they are most likely to differ. + // Also compare timestamps according to millis-since-epoch because otherwise expensive + // comparisons are made on their Chronology objects. + return left.getTimestamp().isEqual(right.getTimestamp()) + && Objects.equals(left.getValue(), right.getValue()) + && Objects.equals(left.getWindows(), right.getWindows()) + && Objects.equals(left.getPaneInfo(), right.getPaneInfo()); } - @Override - public int hashCode() { + public static int hashCode(WindowedValue windowedValue) { // Hash only the millis of the timestamp to be consistent with equals - return Objects.hash(getValue(), getTimestamp().getMillis(), getWindows(), getPane()); + return Objects.hash( + windowedValue.getValue(), + windowedValue.getTimestamp().getMillis(), + windowedValue.getWindows(), + windowedValue.getPane()); } - @Override - public abstract String toString(); - private static final Collection GLOBAL_WINDOWS = Collections.singletonList(GlobalWindow.INSTANCE); - /** A {@link WindowedValue} which holds exactly single window per value. */ + /** A {@link WindowedValues} which holds exactly single window per value. */ public interface SingleWindowedValue { /** @return the single window associated with this value. */ @@ -214,10 +193,10 @@ public interface SingleWindowedValue { } /** - * An abstract superclass for implementations of {@link WindowedValue} that stores the value and + * An abstract superclass for implementations of {@link WindowedValues} that stores the value and * pane info. */ - private abstract static class SimpleWindowedValue extends WindowedValue { + private abstract static class SimpleWindowedValue implements WindowedValue { private final T value; private final PaneInfo pane; @@ -236,6 +215,19 @@ public PaneInfo getPane() { public T getValue() { return value; } + + @Override + public Iterable> explodeWindows() { + if (this.getWindows().size() == 1) { + return ImmutableList.of(this); + } + ImmutableList.Builder> windowedValues = ImmutableList.builder(); + for (BoundedWindow w : this.getWindows()) { + windowedValues.add( + WindowedValues.of(this.getValue(), this.getTimestamp(), w, this.getPaneInfo())); + } + return windowedValues.build(); + } } /** The abstract superclass of WindowedValue representations where timestamp == MIN. */ @@ -258,24 +250,19 @@ public ValueInGlobalWindow(T value, PaneInfo pane) { super(value, pane); } - @Override - public WindowedValue withValue(NewT newValue) { - return new ValueInGlobalWindow<>(newValue, getPane()); - } - @Override public Collection getWindows() { return GLOBAL_WINDOWS; } @Override - public boolean isSingleWindowedValue() { - return true; + public BoundedWindow getWindow() { + return GlobalWindow.INSTANCE; } @Override - public BoundedWindow getWindow() { - return GlobalWindow.INSTANCE; + public WindowedValue withValue(NewT newValue) { + return new ValueInGlobalWindow<>(newValue, getPane()); } @Override @@ -329,24 +316,19 @@ public TimestampedValueInGlobalWindow(T value, Instant timestamp, PaneInfo pane) super(value, timestamp, pane); } - @Override - public WindowedValue withValue(NewT newValue) { - return new TimestampedValueInGlobalWindow<>(newValue, getTimestamp(), getPane()); - } - @Override public Collection getWindows() { return GLOBAL_WINDOWS; } @Override - public boolean isSingleWindowedValue() { - return true; + public BoundedWindow getWindow() { + return GlobalWindow.INSTANCE; } @Override - public BoundedWindow getWindow() { - return GlobalWindow.INSTANCE; + public WindowedValue withValue(NewT newValue) { + return new TimestampedValueInGlobalWindow<>(newValue, getTimestamp(), getPane()); } @Override @@ -405,11 +387,6 @@ public Collection getWindows() { return Collections.singletonList(window); } - @Override - public boolean isSingleWindowedValue() { - return true; - } - @Override public BoundedWindow getWindow() { return window; @@ -459,13 +436,14 @@ public TimestampedValueInMultipleWindows( } @Override - public WindowedValue withValue(NewT newValue) { - return new TimestampedValueInMultipleWindows<>(newValue, getTimestamp(), windows, getPane()); + public Collection getWindows() { + return windows; } @Override - public Collection getWindows() { - return windows; + public WindowedValue withValue(NewT newValue) { + return new TimestampedValueInMultipleWindows<>( + newValue, getTimestamp(), getWindows(), getPane()); } @Override @@ -622,7 +600,7 @@ public WindowedValue decode(InputStream inStream, Context context) // Because there are some remaining (incorrect) uses of WindowedValue with no windows, // we call this deprecated no-validation path when decoding - return WindowedValue.createWithoutValidation(value, timestamp, windows, pane); + return WindowedValues.createWithoutValidation(value, timestamp, windows, pane); } @Override @@ -706,7 +684,7 @@ public WindowedValue decode(InputStream inStream) throws CoderException, IOEx public WindowedValue decode(InputStream inStream, Context context) throws CoderException, IOException { T value = valueCoder.decode(inStream, context); - return WindowedValue.valueInGlobalWindow(value); + return WindowedValues.valueInGlobalWindow(value); } @Override @@ -744,7 +722,7 @@ public static class ParamWindowedValueCoder extends FullWindowedValueCoder /** * Returns the {@link ParamWindowedValueCoder} for the given valueCoder and windowCoder using - * the supplied parameterized timestamp, windows and pane info for {@link WindowedValue}. + * the supplied parameterized timestamp, windows and pane info for {@link WindowedValues}. */ public static ParamWindowedValueCoder of( Coder valueCoder, @@ -787,7 +765,7 @@ public static ParamWindowedValueCoder of(Coder valueCoder) { Collection windows, PaneInfo pane) { super(valueCoder, windowCoder); - this.windowedValuePrototype = WindowedValue.of(EMPTY_BYTES, timestamp, windows, pane); + this.windowedValuePrototype = WindowedValues.of(EMPTY_BYTES, timestamp, windows, pane); } @Override @@ -816,7 +794,7 @@ public WindowedValue decode(InputStream inStream) throws CoderException, IOEx @Override public WindowedValue decode(InputStream inStream, Context context) throws CoderException, IOException { - return windowedValuePrototype.withValue(valueCoder.decode(inStream, context)); + return WindowedValues.withValue(windowedValuePrototype, valueCoder.decode(inStream, context)); } @Override @@ -849,9 +827,9 @@ public static byte[] getPayload(ParamWindowedValueCoder from) { // ParamWindowedValueCoder ByteArrayOutputStream baos = new ByteArrayOutputStream(); WindowedValue windowedValue = - WindowedValue.of(EMPTY_BYTES, from.getTimestamp(), from.getWindows(), from.getPane()); - WindowedValue.FullWindowedValueCoder windowedValueCoder = - WindowedValue.FullWindowedValueCoder.of(ByteArrayCoder.of(), from.getWindowCoder()); + WindowedValues.of(EMPTY_BYTES, from.getTimestamp(), from.getWindows(), from.getPane()); + WindowedValues.FullWindowedValueCoder windowedValueCoder = + WindowedValues.FullWindowedValueCoder.of(ByteArrayCoder.of(), from.getWindowCoder()); try { windowedValueCoder.encode(windowedValue, baos); } catch (IOException e) { @@ -862,17 +840,17 @@ public static byte[] getPayload(ParamWindowedValueCoder from) { } /** Create a {@link Coder} from its component {@link Coder coders}. */ - public static WindowedValue.ParamWindowedValueCoder fromComponents( + public static WindowedValues.ParamWindowedValueCoder fromComponents( List> components, byte[] payload) { Coder windowCoder = (Coder) components.get(1); - WindowedValue.FullWindowedValueCoder windowedValueCoder = - WindowedValue.FullWindowedValueCoder.of(ByteArrayCoder.of(), windowCoder); + WindowedValues.FullWindowedValueCoder windowedValueCoder = + WindowedValues.FullWindowedValueCoder.of(ByteArrayCoder.of(), windowCoder); try { ByteArrayInputStream bais = new ByteArrayInputStream(payload); WindowedValue windowedValue = windowedValueCoder.decode(bais); - return WindowedValue.ParamWindowedValueCoder.of( + return WindowedValues.ParamWindowedValueCoder.of( components.get(0), windowCoder, windowedValue.getTimestamp(), diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserverTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserverTest.java index 91556cf2ac19..f915f9285367 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserverTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserverTest.java @@ -36,7 +36,8 @@ import org.apache.beam.sdk.fn.test.TestExecutors.TestExecutorService; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; @@ -46,7 +47,7 @@ @RunWith(JUnit4.class) public class BeamFnDataInboundObserverTest { private static final Coder> CODER = - WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE); + WindowedValues.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE); private static final String TRANSFORM_ID = "transformId"; private static final String TIMER_FAMILY_ID = "timerFamilyId"; @@ -94,13 +95,13 @@ public void testConsumptionOfValuesHappensOnAwaitCompletionCallersThread() throw assertThat( values, contains( - WindowedValue.valueInGlobalWindow("ABC"), - WindowedValue.valueInGlobalWindow("DEF"), - WindowedValue.valueInGlobalWindow("GHI"))); + WindowedValues.valueInGlobalWindow("ABC"), + WindowedValues.valueInGlobalWindow("DEF"), + WindowedValues.valueInGlobalWindow("GHI"))); assertThat( timers, contains( - WindowedValue.valueInGlobalWindow("UVW"), WindowedValue.valueInGlobalWindow("XYZ"))); + WindowedValues.valueInGlobalWindow("UVW"), WindowedValues.valueInGlobalWindow("XYZ"))); future.get(); } @@ -220,7 +221,7 @@ public void testBadProducerDataFailureVisibleToAwaitCompletionCallerAndProducer( private BeamFnApi.Elements dataWith(String... values) throws Exception { ByteStringOutputStream output = new ByteStringOutputStream(); for (String value : values) { - CODER.encode(WindowedValue.valueInGlobalWindow(value), output); + CODER.encode(WindowedValues.valueInGlobalWindow(value), output); } return BeamFnApi.Elements.newBuilder() .addData( @@ -239,7 +240,7 @@ private BeamFnApi.Elements lastData() throws Exception { private BeamFnApi.Elements timerWith(String... values) throws Exception { ByteStringOutputStream output = new ByteStringOutputStream(); for (String value : values) { - CODER.encode(WindowedValue.valueInGlobalWindow(value), output); + CODER.encode(WindowedValues.valueInGlobalWindow(value), output); } return BeamFnApi.Elements.newBuilder() .addTimers( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java index 85c8d0d04ede..e9cb2664341f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java @@ -62,13 +62,14 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.checkerframework.checker.nullness.qual.Nullable; @@ -347,9 +348,9 @@ public void testCreateTimestampedDefaultOutputCoderUsingTypeDescriptor() throws public void testCreateWindowedValues() { List> data = Arrays.asList( - WindowedValue.of("a", new Instant(1L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), - WindowedValue.of("b", new Instant(2L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of("a", new Instant(1L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), + WindowedValues.of("b", new Instant(2L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), + WindowedValues.of( "c", new Instant(3L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING)); // The easiest way to directly check the created PCollection with PAssert and without relying on diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RedistributeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RedistributeTest.java index 397c71f6beb9..d21eb77819a2 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RedistributeTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RedistributeTest.java @@ -45,13 +45,13 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TypeDescriptors; import org.apache.beam.sdk.values.ValueInSingleWindow; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -180,22 +180,22 @@ public void testRedistributePreservesMetadata() { pipeline .apply( Create.windowedValues( - WindowedValue.of( + WindowedValues.of( "foo", BoundedWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "foo", new Instant(0), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING), - WindowedValue.of( + WindowedValues.of( "bar", new Instant(33), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 1, 1)), - WindowedValue.of( + WindowedValues.of( "bar", GlobalWindow.INSTANCE.maxTimestamp(), GlobalWindow.INSTANCE, diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java index 4df02ea0cf59..099dcb78a6e0 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java @@ -48,12 +48,12 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TypeDescriptors; import org.apache.beam.sdk.values.ValueInSingleWindow; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -171,22 +171,22 @@ public void testReshufflePreservesMetadata() { pipeline .apply( Create.windowedValues( - WindowedValue.of( + WindowedValues.of( "foo", BoundedWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "foo", new Instant(0), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING), - WindowedValue.of( + WindowedValues.of( "bar", new Instant(33), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 1, 1)), - WindowedValue.of( + WindowedValues.of( "bar", GlobalWindow.INSTANCE.maxTimestamp(), GlobalWindow.INSTANCE, diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java index 0a18f076762a..db1579333e57 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java @@ -20,6 +20,8 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.not; import java.util.Arrays; import org.apache.beam.sdk.coders.Coder; @@ -31,6 +33,8 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.joda.time.Duration; @@ -52,7 +56,7 @@ public class WindowedValueTest { public void testWindowedValueCoder() throws CoderException { Instant timestamp = new Instant(1234); WindowedValue value = - WindowedValue.of( + WindowedValues.of( "abc", new Instant(1234), Arrays.asList( @@ -62,7 +66,7 @@ public void testWindowedValueCoder() throws CoderException { PaneInfo.NO_FIRING); Coder> windowedValueCoder = - WindowedValue.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()); + WindowedValues.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()); byte[] encodedValue = CoderUtils.encodeToByteArray(windowedValueCoder, value); WindowedValue decodedValue = @@ -76,24 +80,25 @@ public void testWindowedValueCoder() throws CoderException { @Test public void testFullWindowedValueCoderIsSerializableWithWellKnownCoderType() { CoderProperties.coderSerializable( - WindowedValue.getFullCoder(GlobalWindow.Coder.INSTANCE, GlobalWindow.Coder.INSTANCE)); + WindowedValues.getFullCoder(GlobalWindow.Coder.INSTANCE, GlobalWindow.Coder.INSTANCE)); } @Test public void testParamWindowedValueCoderIsSerializableWithWellKnownCoderType() { CoderProperties.coderSerializable( - WindowedValue.getParamWindowedValueCoder(GlobalWindow.Coder.INSTANCE)); + WindowedValues.getParamWindowedValueCoder(GlobalWindow.Coder.INSTANCE)); } @Test public void testValueOnlyWindowedValueCoderIsSerializableWithWellKnownCoderType() { - CoderProperties.coderSerializable(WindowedValue.getValueOnlyCoder(GlobalWindow.Coder.INSTANCE)); + CoderProperties.coderSerializable( + WindowedValues.getValueOnlyCoder(GlobalWindow.Coder.INSTANCE)); } @Test public void testExplodeWindowsInNoWindowsCrash() { thrown.expect(IllegalArgumentException.class); - WindowedValue.of("foo", Instant.now(), ImmutableList.of(), PaneInfo.NO_FIRING); + WindowedValues.of("foo", Instant.now(), ImmutableList.of(), PaneInfo.NO_FIRING); } @Test @@ -102,7 +107,7 @@ public void testExplodeWindowsInOneWindowEquals() { BoundedWindow window = new IntervalWindow(now.minus(Duration.millis(1000L)), now.plus(Duration.millis(1000L))); WindowedValue value = - WindowedValue.of("foo", now, window, PaneInfo.ON_TIME_AND_ONLY_FIRING); + WindowedValues.of("foo", now, window, PaneInfo.ON_TIME_AND_ONLY_FIRING); assertThat(Iterables.getOnlyElement(value.explodeWindows()), equalTo(value)); } @@ -119,7 +124,7 @@ public void testExplodeWindowsManyWindowsMultipleWindowedValues() { BoundedWindow futureFutureWindow = new IntervalWindow(now, now.plus(Duration.millis(2000L))); PaneInfo pane = PaneInfo.createPane(false, false, Timing.ON_TIME, 3L, 0L); WindowedValue value = - WindowedValue.of( + WindowedValues.of( "foo", now, ImmutableList.of(pastWindow, centerWindow, futureWindow, futureFutureWindow), @@ -128,29 +133,29 @@ public void testExplodeWindowsManyWindowsMultipleWindowedValues() { assertThat( value.explodeWindows(), containsInAnyOrder( - WindowedValue.of("foo", now, futureFutureWindow, pane), - WindowedValue.of("foo", now, futureWindow, pane), - WindowedValue.of("foo", now, centerWindow, pane), - WindowedValue.of("foo", now, pastWindow, pane))); + WindowedValues.of("foo", now, futureFutureWindow, pane), + WindowedValues.of("foo", now, futureWindow, pane), + WindowedValues.of("foo", now, centerWindow, pane), + WindowedValues.of("foo", now, pastWindow, pane))); - assertThat(value.isSingleWindowedValue(), equalTo(false)); + assertThat(value, not(instanceOf(WindowedValues.SingleWindowedValue.class))); } @Test public void testSingleWindowedValueInGlobalWindow() { WindowedValue value = - WindowedValue.of(1, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); - assertThat(value.isSingleWindowedValue(), equalTo(true)); + WindowedValues.of(1, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); + assertThat(value, instanceOf(WindowedValues.SingleWindowedValue.class)); assertThat( - ((WindowedValue.SingleWindowedValue) value).getWindow(), equalTo(GlobalWindow.INSTANCE)); + ((WindowedValues.SingleWindowedValue) value).getWindow(), equalTo(GlobalWindow.INSTANCE)); } @Test public void testSingleWindowedValueInFixedWindow() { Instant now = Instant.now(); BoundedWindow w = new IntervalWindow(now, now.plus(Duration.millis(1))); - WindowedValue value = WindowedValue.of(1, now, w, PaneInfo.NO_FIRING); - assertThat(value.isSingleWindowedValue(), equalTo(true)); - assertThat(((WindowedValue.SingleWindowedValue) value).getWindow(), equalTo(w)); + WindowedValue value = WindowedValues.of(1, now, w, PaneInfo.NO_FIRING); + assertThat(value, instanceOf(WindowedValues.SingleWindowedValue.class)); + assertThat(((WindowedValues.SingleWindowedValue) value).getWindow(), equalTo(w)); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/CoderTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/CoderTranslationTest.java index 7dbe544d5948..b8f92ff0053e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/CoderTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/CoderTranslationTest.java @@ -52,9 +52,9 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder; import org.apache.beam.sdk.util.ShardedKey; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.construction.CoderTranslation.TranslationContext; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.hamcrest.Matchers; @@ -85,7 +85,7 @@ public class CoderTranslationTest { .add( FullWindowedValueCoder.of( IterableCoder.of(VarLongCoder.of()), IntervalWindowCoder.of())) - .add(WindowedValue.ParamWindowedValueCoder.of(IterableCoder.of(VarLongCoder.of()))) + .add(WindowedValues.ParamWindowedValueCoder.of(IterableCoder.of(VarLongCoder.of()))) .add(DoubleCoder.of()) .add( RowCoder.of( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ModelCodersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ModelCodersTest.java index f0a277760ac4..b21368f9fc30 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ModelCodersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ModelCodersTest.java @@ -32,10 +32,10 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.construction.ModelCoders.KvCoderComponents; import org.apache.beam.sdk.util.construction.ModelCoders.WindowedValueCoderComponents; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/WindowingTest.java b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/WindowingTest.java index 80eb878de9c8..1b4179ff9e9c 100644 --- a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/WindowingTest.java +++ b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/WindowingTest.java @@ -16,6 +16,7 @@ * limitations under the License. */ package org.apache.beam.sdk.extensions.euphoria.core.testkit; // + // import static org.junit.Assert.assertEquals; // // import java.time.Instant; diff --git a/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/PrecombineGroupingTableBenchmark.java b/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/PrecombineGroupingTableBenchmark.java index ded8358a10d4..54f773d402cc 100644 --- a/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/PrecombineGroupingTableBenchmark.java +++ b/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/PrecombineGroupingTableBenchmark.java @@ -30,8 +30,9 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Sum; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Level; import org.openjdk.jmh.annotations.Param; @@ -74,13 +75,13 @@ private static List>> generateTestData(String case "uniform": for (int i = 0; i < TOTAL_VALUES; ++i) { int key = random.nextInt(KEY_SPACE); - elements.add(WindowedValue.valueInGlobalWindow(KV.of(Integer.toString(key), key))); + elements.add(WindowedValues.valueInGlobalWindow(KV.of(Integer.toString(key), key))); } break; case "normal": for (int i = 0; i < TOTAL_VALUES; ++i) { int key = (int) (random.nextGaussian() * KEY_SPACE); - elements.add(WindowedValue.valueInGlobalWindow(KV.of(Integer.toString(key), key))); + elements.add(WindowedValues.valueInGlobalWindow(KV.of(Integer.toString(key), key))); } break; case "hotKey": @@ -91,12 +92,12 @@ private static List>> generateTestData(String } else { key = random.nextInt(KEY_SPACE); } - elements.add(WindowedValue.valueInGlobalWindow(KV.of(Integer.toString(key), key))); + elements.add(WindowedValues.valueInGlobalWindow(KV.of(Integer.toString(key), key))); } break; case "uniqueKeys": for (int i = 0; i < TOTAL_VALUES; ++i) { - elements.add(WindowedValue.valueInGlobalWindow(KV.of(Integer.toString(i), i))); + elements.add(WindowedValues.valueInGlobalWindow(KV.of(Integer.toString(i), i))); } Collections.shuffle(elements, random); break; diff --git a/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/ProcessBundleBenchmark.java b/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/ProcessBundleBenchmark.java index 42c288673734..3b8fbeaf3dd0 100644 --- a/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/ProcessBundleBenchmark.java +++ b/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/ProcessBundleBenchmark.java @@ -17,7 +17,7 @@ */ package org.apache.beam.fn.harness.jmh; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import static org.junit.Assert.assertEquals; @@ -83,13 +83,13 @@ import org.apache.beam.sdk.transforms.Impulse; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.WithKeys; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.BeamUrns; import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.sdk.util.construction.graph.FusedPipeline; import org.apache.beam.sdk.util.construction.graph.GreedyPipelineFuser; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/AssignWindowsRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/AssignWindowsRunner.java index 8e4f6a2b2036..957fa057054f 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/AssignWindowsRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/AssignWindowsRunner.java @@ -30,9 +30,10 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -109,6 +110,6 @@ public BoundedWindow window() { } }; Collection windows = windowFn.assignWindows(ctxt); - return WindowedValue.of(input.getValue(), input.getTimestamp(), windows, input.getPane()); + return WindowedValues.of(input.getValue(), input.getTimestamp(), windows, input.getPane()); } } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java index 834d78e28a57..8e1231347a1e 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java @@ -48,9 +48,9 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.fn.data.RemoteGrpcPortRead; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CoderTranslation; import org.apache.beam.sdk.util.construction.RehydratedComponents; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java index 7995961243ea..78d13d14bf68 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java @@ -31,9 +31,9 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.fn.data.RemoteGrpcPortWrite; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CoderTranslation; import org.apache.beam.sdk.util.construction.RehydratedComponents; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; /** diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java index 9ac7f6de5476..718fe7ee8b67 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java @@ -35,11 +35,11 @@ import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FlattenRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FlattenRunner.java index c0986010c635..d79a79ada540 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FlattenRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FlattenRunner.java @@ -23,8 +23,8 @@ import java.io.IOException; import java.util.Map; import org.apache.beam.sdk.fn.data.FnDataReceiver; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; /** Executes flatten PTransforms. */ diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java index e264fa14788a..460c4cc6cab7 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java @@ -99,8 +99,6 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.util.construction.RehydratedComponents; @@ -109,6 +107,9 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.util.Durations; @@ -376,7 +377,7 @@ public final void addRunnerForPTransform(Context context) throws IOException { components.getPcollectionsMap().get(pTransform.getInputsOrThrow(mainInputTag)); Coder maybeWindowedValueInputCoder = rehydratedComponents.getCoder(mainInput.getCoderId()); // TODO: Stop passing windowed value coders within PCollections. - if (maybeWindowedValueInputCoder instanceof WindowedValue.WindowedValueCoder) { + if (maybeWindowedValueInputCoder instanceof WindowedValues.WindowedValueCoder) { inputCoder = ((WindowedValueCoder) maybeWindowedValueInputCoder).getValueCoder(); } else { inputCoder = maybeWindowedValueInputCoder; @@ -1011,28 +1012,28 @@ public Object restriction() { return WindowedSplitResult.forRoots( splitResult.getPrimaryInFullyProcessedWindowsRoot() == null ? null - : WindowedValue.of( + : WindowedValues.of( KV.of(splitResult.getPrimaryInFullyProcessedWindowsRoot().getValue(), fullSize), splitResult.getPrimaryInFullyProcessedWindowsRoot().getTimestamp(), splitResult.getPrimaryInFullyProcessedWindowsRoot().getWindows(), splitResult.getPrimaryInFullyProcessedWindowsRoot().getPane()), splitResult.getPrimarySplitRoot() == null ? null - : WindowedValue.of( + : WindowedValues.of( KV.of(splitResult.getPrimarySplitRoot().getValue(), primarySize), splitResult.getPrimarySplitRoot().getTimestamp(), splitResult.getPrimarySplitRoot().getWindows(), splitResult.getPrimarySplitRoot().getPane()), splitResult.getResidualSplitRoot() == null ? null - : WindowedValue.of( + : WindowedValues.of( KV.of(splitResult.getResidualSplitRoot().getValue(), residualSize), splitResult.getResidualSplitRoot().getTimestamp(), splitResult.getResidualSplitRoot().getWindows(), splitResult.getResidualSplitRoot().getPane()), splitResult.getResidualInUnprocessedWindowsRoot() == null ? null - : WindowedValue.of( + : WindowedValues.of( KV.of(splitResult.getResidualInUnprocessedWindowsRoot().getValue(), fullSize), splitResult.getResidualInUnprocessedWindowsRoot().getTimestamp(), splitResult.getResidualInUnprocessedWindowsRoot().getWindows(), @@ -1081,7 +1082,7 @@ private HandlesSplits.SplitResult trySplitForWindowObservingTruncateRestriction( } // Note that the assumption here is the fullInputCoder of the Truncate transform should be the // the same as the SDF/Process transform. - Coder fullInputCoder = WindowedValue.getFullCoder(inputCoder, windowCoder); + Coder fullInputCoder = WindowedValues.getFullCoder(inputCoder, windowCoder); return constructSplitResult( windowedSplitResult, downstreamSplitResult, @@ -1113,7 +1114,7 @@ private static WindowedSplitResult computeWindowSplit WindowedSplitResult.forRoots( primaryFullyProcessedWindows.isEmpty() ? null - : WindowedValue.of( + : WindowedValues.of( KV.of( currentElement.getValue(), KV.of(currentRestriction, currentWatermarkEstimatorState)), @@ -1122,7 +1123,7 @@ private static WindowedSplitResult computeWindowSplit currentElement.getPane()), splitResult == null ? null - : WindowedValue.of( + : WindowedValues.of( KV.of( currentElement.getValue(), KV.of(splitResult.getPrimary(), currentWatermarkEstimatorState)), @@ -1131,7 +1132,7 @@ private static WindowedSplitResult computeWindowSplit currentElement.getPane()), splitResult == null ? null - : WindowedValue.of( + : WindowedValues.of( KV.of( currentElement.getValue(), KV.of(splitResult.getResidual(), watermarkAndState.getValue())), @@ -1140,7 +1141,7 @@ private static WindowedSplitResult computeWindowSplit currentElement.getPane()), residualUnprocessedWindows.isEmpty() ? null - : WindowedValue.of( + : WindowedValues.of( KV.of( currentElement.getValue(), KV.of(currentRestriction, currentWatermarkEstimatorState)), @@ -1439,7 +1440,7 @@ private HandlesSplits.SplitResult trySplitForElementAndRestriction( PTransformTranslation.SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN + "/GetSize"); } - Coder fullInputCoder = WindowedValue.getFullCoder(inputCoder, windowCoder); + Coder fullInputCoder = WindowedValues.getFullCoder(inputCoder, windowCoder); return constructSplitResult( windowedSplitResult, null, @@ -1913,7 +1914,7 @@ public PipelineOptions getPipelineOptions() { @Override public void output(OutputT output, Instant timestamp, BoundedWindow window) { outputTo( - mainOutputConsumer, WindowedValue.of(output, timestamp, window, PaneInfo.NO_FIRING)); + mainOutputConsumer, WindowedValues.of(output, timestamp, window, PaneInfo.NO_FIRING)); } @Override @@ -1923,7 +1924,7 @@ public void output(TupleTag tag, T output, Instant timestamp, BoundedWind if (consumer == null) { throw new IllegalArgumentException(String.format("Unknown output tag %s", tag)); } - outputTo(consumer, WindowedValue.of(output, timestamp, window, PaneInfo.NO_FIRING)); + outputTo(consumer, WindowedValues.of(output, timestamp, window, PaneInfo.NO_FIRING)); } } @@ -1978,7 +1979,7 @@ public void output(OutputT output) { // Don't need to check timestamp since we can always output using the input timestamp. outputTo( mainOutputConsumer, - WindowedValue.of( + WindowedValues.of( output, currentElement.getTimestamp(), currentWindow, currentElement.getPane())); } @@ -1992,7 +1993,7 @@ public void output(TupleTag tag, T output) { // Don't need to check timestamp since we can always output using the input timestamp. outputTo( consumer, - WindowedValue.of( + WindowedValues.of( output, currentElement.getTimestamp(), currentWindow, currentElement.getPane())); } @@ -2002,7 +2003,7 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { // runners can provide proper timestamps. outputTo( mainOutputConsumer, - WindowedValue.of(output, timestamp, currentWindow, currentElement.getPane())); + WindowedValues.of(output, timestamp, currentWindow, currentElement.getPane())); } @Override @@ -2013,7 +2014,7 @@ public void outputWindowedValue( PaneInfo paneInfo) { // TODO(https://github.com/apache/beam/issues/29637): Check that timestamp is valid once all // runners can provide proper timestamps. - outputTo(mainOutputConsumer, WindowedValue.of(output, timestamp, windows, paneInfo)); + outputTo(mainOutputConsumer, WindowedValues.of(output, timestamp, windows, paneInfo)); } @Override @@ -2026,7 +2027,7 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp throw new IllegalArgumentException(String.format("Unknown output tag %s", tag)); } outputTo( - consumer, WindowedValue.of(output, timestamp, currentWindow, currentElement.getPane())); + consumer, WindowedValues.of(output, timestamp, currentWindow, currentElement.getPane())); } @Override @@ -2043,7 +2044,7 @@ public void outputWindowedValue( if (consumer == null) { throw new IllegalArgumentException(String.format("Unknown output tag %s", tag)); } - outputTo(consumer, WindowedValue.of(output, timestamp, windows, paneInfo)); + outputTo(consumer, WindowedValues.of(output, timestamp, windows, paneInfo)); } @Override @@ -2135,7 +2136,7 @@ public Instant timestamp(DoFn doFn) { outputTo( mainOutputConsumer, (WindowedValue) - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( currentElement.getValue(), KV.of(output, currentWatermarkEstimatorState)), @@ -2180,7 +2181,7 @@ public Instant timestamp(DoFn doFn) { outputTo( mainOutputConsumer, (WindowedValue) - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( currentElement.getValue(), KV.of(output, currentWatermarkEstimatorState)), @@ -2220,7 +2221,7 @@ public Instant timestamp(DoFn doFn) { outputTo( mainOutputConsumer, (WindowedValue) - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( currentElement.getValue(), KV.of(output, currentWatermarkEstimatorState)), @@ -2348,7 +2349,7 @@ public Instant timestamp(DoFn doFn) { outputTo( mainOutputConsumer, (WindowedValue) - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( currentElement.getValue(), KV.of(output, currentWatermarkEstimatorState)), @@ -2388,7 +2389,7 @@ public Instant timestamp(DoFn doFn) { outputTo( mainOutputConsumer, (WindowedValue) - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( currentElement.getValue(), KV.of(output, currentWatermarkEstimatorState)), @@ -2450,7 +2451,7 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { checkTimestamp(timestamp); outputTo( mainOutputConsumer, - WindowedValue.of( + WindowedValues.of( output, timestamp, currentElement.getWindows(), currentElement.getPane())); } @@ -2461,7 +2462,7 @@ public void outputWindowedValue( Collection windows, PaneInfo paneInfo) { checkTimestamp(timestamp); - outputTo(mainOutputConsumer, WindowedValue.of(output, timestamp, windows, paneInfo)); + outputTo(mainOutputConsumer, WindowedValues.of(output, timestamp, windows, paneInfo)); } @Override @@ -2474,7 +2475,7 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp } outputTo( consumer, - WindowedValue.of( + WindowedValues.of( output, timestamp, currentElement.getWindows(), currentElement.getPane())); } @@ -2491,7 +2492,7 @@ public void outputWindowedValue( if (consumer == null) { throw new IllegalArgumentException(String.format("Unknown output tag %s", tag)); } - outputTo(consumer, WindowedValue.of(output, timestamp, windows, paneInfo)); + outputTo(consumer, WindowedValues.of(output, timestamp, windows, paneInfo)); } } @@ -2824,7 +2825,7 @@ public BoundedWindow window() { public void output(OutputT output) { outputTo( mainOutputConsumer, - WindowedValue.of( + WindowedValues.of( output, currentTimer.getHoldTimestamp(), currentWindow, currentTimer.getPane())); } @@ -2833,7 +2834,7 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { checkOnWindowExpirationTimestamp(timestamp); outputTo( mainOutputConsumer, - WindowedValue.of(output, timestamp, currentWindow, currentTimer.getPane())); + WindowedValues.of(output, timestamp, currentWindow, currentTimer.getPane())); } @Override @@ -2843,7 +2844,7 @@ public void outputWindowedValue( Collection windows, PaneInfo paneInfo) { checkOnWindowExpirationTimestamp(timestamp); - outputTo(mainOutputConsumer, WindowedValue.of(output, timestamp, windows, paneInfo)); + outputTo(mainOutputConsumer, WindowedValues.of(output, timestamp, windows, paneInfo)); } @Override @@ -2855,7 +2856,7 @@ public void output(TupleTag tag, T output) { } outputTo( consumer, - WindowedValue.of( + WindowedValues.of( output, currentTimer.getHoldTimestamp(), currentWindow, currentTimer.getPane())); } @@ -2868,7 +2869,7 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp throw new IllegalArgumentException(String.format("Unknown output tag %s", tag)); } outputTo( - consumer, WindowedValue.of(output, timestamp, currentWindow, currentTimer.getPane())); + consumer, WindowedValues.of(output, timestamp, currentWindow, currentTimer.getPane())); } @Override @@ -2881,7 +2882,7 @@ public void outputWindowedValue( checkOnWindowExpirationTimestamp(timestamp); FnDataReceiver> consumer = (FnDataReceiver) localNameToConsumer.get(tag.getId()); - outputTo(consumer, WindowedValue.of(output, timestamp, windows, paneInfo)); + outputTo(consumer, WindowedValues.of(output, timestamp, windows, paneInfo)); } @SuppressWarnings( @@ -3120,7 +3121,7 @@ public void output(OutputT output) { checkTimerTimestamp(currentTimer.getHoldTimestamp()); outputTo( mainOutputConsumer, - WindowedValue.of( + WindowedValues.of( output, currentTimer.getHoldTimestamp(), currentWindow, currentTimer.getPane())); } @@ -3129,7 +3130,7 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { checkTimerTimestamp(timestamp); outputTo( mainOutputConsumer, - WindowedValue.of(output, timestamp, currentWindow, currentTimer.getPane())); + WindowedValues.of(output, timestamp, currentWindow, currentTimer.getPane())); } @Override @@ -3139,7 +3140,7 @@ public void outputWindowedValue( Collection windows, PaneInfo paneInfo) { checkTimerTimestamp(timestamp); - outputTo(mainOutputConsumer, WindowedValue.of(output, timestamp, windows, paneInfo)); + outputTo(mainOutputConsumer, WindowedValues.of(output, timestamp, windows, paneInfo)); } @Override @@ -3152,7 +3153,7 @@ public void output(TupleTag tag, T output) { } outputTo( consumer, - WindowedValue.of( + WindowedValues.of( output, currentTimer.getHoldTimestamp(), currentWindow, currentTimer.getPane())); } @@ -3165,7 +3166,7 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp throw new IllegalArgumentException(String.format("Unknown output tag %s", tag)); } outputTo( - consumer, WindowedValue.of(output, timestamp, currentWindow, currentTimer.getPane())); + consumer, WindowedValues.of(output, timestamp, currentWindow, currentTimer.getPane())); } @Override diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/MapFnRunners.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/MapFnRunners.java index ddd6faa46d16..dbc9fc05020a 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/MapFnRunners.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/MapFnRunners.java @@ -23,7 +23,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.function.ThrowingFunction; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; /** diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java index 7b53f6511045..8e7c670d5c26 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java @@ -36,8 +36,8 @@ import org.apache.beam.sdk.function.ThrowingRunnable; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Timer; +import org.apache.beam.sdk.values.WindowedValue; /** A factory able to instantiate an appropriate handler for a given PTransform. */ public interface PTransformRunnerFactory { diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PrecombineGroupingTable.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PrecombineGroupingTable.java index 06cc17062bdb..d6827e821782 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PrecombineGroupingTable.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PrecombineGroupingTable.java @@ -37,8 +37,9 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.Weighted; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.joda.time.Instant; @@ -468,8 +469,8 @@ private void output( entry.compact(); receiver.accept( isGloballyWindowed - ? WindowedValue.valueInGlobalWindow(KV.of(entry.getKey(), entry.getAccumulator())) - : WindowedValue.of( + ? WindowedValues.valueInGlobalWindow(KV.of(entry.getKey(), entry.getAccumulator())) + : WindowedValues.of( KV.of(entry.getKey(), entry.getAccumulator()), entry.getOutputTimestamp(), entry.getGroupingKey().getWindows(), diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunner.java index f19e98ee9e73..72565f608ce5 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunner.java @@ -39,12 +39,13 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.checkerframework.checker.nullness.qual.Nullable; @@ -203,7 +204,7 @@ private void processElementForWindowObservingPairWithRestriction(WindowedValue> getMultiplexingConsumer(String pCollecti } // TODO: Stop passing windowed value coders within PCollections. - if (maybeWindowedValueInputCoder instanceof WindowedValue.WindowedValueCoder) { + if (maybeWindowedValueInputCoder instanceof WindowedValues.WindowedValueCoder) { coder = ((WindowedValueCoder) maybeWindowedValueInputCoder).getValueCoder(); } else { coder = maybeWindowedValueInputCoder; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/ElementSample.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/ElementSample.java index 4ef1674c9ec6..180e9634500f 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/ElementSample.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/ElementSample.java @@ -18,7 +18,7 @@ package org.apache.beam.fn.harness.debug; import javax.annotation.Nullable; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** * A record class that wraps an element sample with additional metadata. This ensures the ability to diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java index f7fabae0cc21..f526f1d01110 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java @@ -29,7 +29,8 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; /** * This class holds samples for a single PCollection until queried by the parent DataSampler. This @@ -79,7 +80,7 @@ public OutputSampler( // element itself is sampled. Or, it's non a WindowedValueCoder and the value inside the // windowed value must be sampled. This is because WindowedValue is the element type used in // all receivers, which doesn't necessarily match the PBD encoding. - if (coder instanceof WindowedValue.WindowedValueCoder) { + if (coder instanceof WindowedValues.WindowedValueCoder) { this.valueCoder = null; this.windowedValueCoder = (Coder>) coder; } else { diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java index 2b9d8b79f113..164589bc40aa 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java @@ -65,13 +65,13 @@ import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.CombineFnUtil; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.BeamUrns; import org.apache.beam.sdk.util.construction.PCollectionViewTranslation; import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -189,9 +189,9 @@ public static Factory factoryForPTransformContext( .get(context.getPTransform().getInputsOrThrow(mainInputTag)); Coder maybeWindowedValueInputCoder = rehydratedComponents.getCoder(mainInput.getCoderId()); Coder inputCoder; - if (maybeWindowedValueInputCoder instanceof WindowedValue.WindowedValueCoder) { + if (maybeWindowedValueInputCoder instanceof WindowedValues.WindowedValueCoder) { inputCoder = - ((WindowedValue.WindowedValueCoder) maybeWindowedValueInputCoder).getValueCoder(); + ((WindowedValues.WindowedValueCoder) maybeWindowedValueInputCoder).getValueCoder(); } else { inputCoder = maybeWindowedValueInputCoder; } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java index 838715d1beda..9fbf21683b63 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java @@ -44,12 +44,13 @@ import org.apache.beam.sdk.transforms.windowing.SlidingWindows; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CoderTranslation; import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.joda.time.Duration; @@ -74,22 +75,22 @@ public void singleInputSingleOutputSucceeds() throws Exception { AssignWindowsRunner runner = AssignWindowsRunner.create(windowFn); assertThat( - runner.assignWindows(WindowedValue.valueInGlobalWindow(1)), + runner.assignWindows(WindowedValues.valueInGlobalWindow(1)), equalTo( - WindowedValue.of( + WindowedValues.of( 1, BoundedWindow.TIMESTAMP_MIN_VALUE, windowFn.assignWindow(BoundedWindow.TIMESTAMP_MIN_VALUE), PaneInfo.NO_FIRING))); assertThat( runner.assignWindows( - WindowedValue.of( + WindowedValues.of( 2, new Instant(-10L), new IntervalWindow(new Instant(-120000L), Duration.standardMinutes(3L)), PaneInfo.ON_TIME_AND_ONLY_FIRING)), equalTo( - WindowedValue.of( + WindowedValues.of( 2, new Instant(-10L), windowFn.assignWindow(new Instant(-10L)), @@ -112,17 +113,17 @@ public void singleInputMultipleOutputSucceeds() throws Exception { IntervalWindow thirdWindow = new IntervalWindow(new Instant(0), Duration.standardMinutes(4L)); WindowedValue firstValue = - WindowedValue.timestampedValueInGlobalWindow(-3, new Instant(-12)); + WindowedValues.timestampedValueInGlobalWindow(-3, new Instant(-12)); assertThat( runner.assignWindows(firstValue), equalTo( - WindowedValue.of( + WindowedValues.of( -3, new Instant(-12), ImmutableSet.of(firstWindow, secondWindow), firstValue.getPane()))); WindowedValue secondValue = - WindowedValue.of( + WindowedValues.of( 3, new Instant(12), new IntervalWindow(new Instant(-12), Duration.standardMinutes(24)), @@ -131,7 +132,7 @@ public void singleInputMultipleOutputSucceeds() throws Exception { assertThat( runner.assignWindows(secondValue), equalTo( - WindowedValue.of( + WindowedValues.of( 3, new Instant(12), ImmutableSet.of(secondWindow, thirdWindow), @@ -205,7 +206,7 @@ public Coder windowCoder() { .addRunnerForPTransform(context); WindowedValue value = - WindowedValue.of( + WindowedValues.of( 2, new Instant(-10L), ImmutableList.of( @@ -216,14 +217,14 @@ public Coder windowCoder() { assertThat( outputs, containsInAnyOrder( - WindowedValue.of( + WindowedValues.of( 2, new Instant(-10L), ImmutableSet.of( GlobalWindow.INSTANCE, new IntervalWindow(new Instant(-500), Duration.standardMinutes(3))), PaneInfo.ON_TIME_AND_ONLY_FIRING), - WindowedValue.of( + WindowedValues.of( 2, new Instant(-10L), ImmutableSet.of( @@ -265,7 +266,7 @@ public Coder windowCoder() { thrown.expect(IllegalArgumentException.class); runner.assignWindows( - WindowedValue.of( + WindowedValues.of( 2, new Instant(-10L), ImmutableList.of( @@ -300,13 +301,13 @@ public void factoryCreatesFromJavaWindowFn() throws Exception { assertThat( fn.apply( - WindowedValue.of( + WindowedValues.of( 22L, new Instant(5), new IntervalWindow(new Instant(0L), new Instant(20027L)), PaneInfo.ON_TIME_AND_ONLY_FIRING)), equalTo( - WindowedValue.of( + WindowedValues.of( 22L, new Instant(5), new TestWindowFn().assignWindow(new Instant(5)), @@ -338,7 +339,7 @@ public void factoryCreatesFromKnownWindowFn() throws Exception { (ThrowingFunction) factory.forPTransform("transform", windowPTransform); WindowedValue output = fn.apply( - WindowedValue.of( + WindowedValues.of( 22L, new Instant(5), new IntervalWindow(new Instant(0L), new Instant(20027L)), @@ -347,7 +348,7 @@ public void factoryCreatesFromKnownWindowFn() throws Exception { assertThat( output, equalTo( - WindowedValue.of( + WindowedValues.of( 22L, new Instant(5), new IntervalWindow(new Instant(5L), Duration.standardMinutes(12L)), diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java index 6ec40c10bfb2..40ebebacd285 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.fn.harness; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -65,8 +65,9 @@ import org.apache.beam.sdk.fn.test.TestExecutors; import org.apache.beam.sdk.fn.test.TestExecutors.TestExecutorService; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CoderTranslation; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @@ -88,7 +89,7 @@ public class BeamFnDataReadRunnerTest { private static final Coder ELEMENT_CODER = StringUtf8Coder.of(); private static final String ELEMENT_CODER_SPEC_ID = "string-coder-id"; private static final Coder> CODER = - WindowedValue.getFullCoder(ELEMENT_CODER, GlobalWindow.Coder.INSTANCE); + WindowedValues.getFullCoder(ELEMENT_CODER, GlobalWindow.Coder.INSTANCE); private static final String CODER_SPEC_ID = "windowed-string-coder-id"; private static final RunnerApi.Coder CODER_SPEC; private static final RunnerApi.Components COMPONENTS; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java index f9b03ba4b0a9..70a894e7b375 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.fn.harness; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -51,8 +51,9 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CoderTranslation; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.hamcrest.collection.IsMapContaining; @@ -71,7 +72,7 @@ public class BeamFnDataWriteRunnerTest { private static final Coder ELEM_CODER = StringUtf8Coder.of(); private static final String WIRE_CODER_ID = "windowed-string-coder-id"; private static final Coder> WIRE_CODER = - WindowedValue.getFullCoder(ELEM_CODER, GlobalWindow.Coder.INSTANCE); + WindowedValues.getFullCoder(ELEM_CODER, GlobalWindow.Coder.INSTANCE); private static final RunnerApi.Coder WIRE_CODER_SPEC; private static final RunnerApi.Components COMPONENTS; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/CombineRunnersTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/CombineRunnersTest.java index 6c2b7cb20ace..fc3cad970f73 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/CombineRunnersTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/CombineRunnersTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.fn.harness; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -38,12 +38,12 @@ import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.ModelCoders; import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.junit.Before; import org.junit.Test; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FlattenRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FlattenRunnerTest.java index 98bd85b4005f..0a839588e537 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FlattenRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FlattenRunnerTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.fn.harness; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -32,9 +32,10 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.fn.data.FnDataReceiver; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CoderTranslation; import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -175,8 +176,8 @@ public void testFlattenWithDuplicateInputCollectionProducesMultipleOutputs() thr FnDataReceiver> input = context.getPCollectionConsumer("inputATarget"); - input.accept(WindowedValue.valueInGlobalWindow("A1")); - input.accept(WindowedValue.valueInGlobalWindow("A2")); + input.accept(WindowedValues.valueInGlobalWindow("A1")); + input.accept(WindowedValues.valueInGlobalWindow("A2")); assertThat( mainOutputValues, diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java index 6ca085495a3d..b9c879ad827a 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java @@ -19,8 +19,8 @@ import static java.util.Arrays.asList; import static org.apache.beam.sdk.options.ExperimentalOptions.addExperiment; -import static org.apache.beam.sdk.util.WindowedValue.timestampedValueInGlobalWindow; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.timestampedValueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.allOf; @@ -116,7 +116,6 @@ import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CoderTranslation; import org.apache.beam.sdk.util.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.util.construction.PTransformTranslation; @@ -132,6 +131,8 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.util.Durations; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -1166,7 +1167,7 @@ private org.apache.beam.sdk.util.construction.Timer dynamicTimerInGlobalW private WindowedValue valueInWindows( T value, BoundedWindow window, BoundedWindow... windows) { - return WindowedValue.of( + return WindowedValues.of( value, window.maxTimestamp(), ImmutableList.builder().add(window).add(windows).build(), @@ -1413,7 +1414,7 @@ public void testProcessElementForSizedElementAndRestriction() throws Exception { RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(pProto.getComponents()); Coder inputCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( CoderTranslation.fromProto( pProto.getComponents().getCodersOrThrow(inputPCollection.getCoderId()), rehydratedComponents, @@ -1707,7 +1708,7 @@ public void testProcessElementForSizedElementAndRestrictionSplitBeforeTryClaim() RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(pProto.getComponents()); Coder inputCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( CoderTranslation.fromProto( pProto.getComponents().getCodersOrThrow(inputPCollection.getCoderId()), rehydratedComponents, @@ -1921,7 +1922,7 @@ public void testProcessElementForSizedElementAndRestrictionNoTryClaim() throws E RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(pProto.getComponents()); Coder inputCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( CoderTranslation.fromProto( pProto.getComponents().getCodersOrThrow(inputPCollection.getCoderId()), rehydratedComponents, @@ -2120,7 +2121,7 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(pProto.getComponents()); Coder inputCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( CoderTranslation.fromProto( pProto.getComponents().getCodersOrThrow(inputPCollection.getCoderId()), rehydratedComponents, @@ -2256,7 +2257,7 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc assertEquals( decode(inputCoder, primaryRoot.getElement()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( "5", @@ -2269,7 +2270,7 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc firstValue.getPane())); assertEquals( decode(inputCoder, residualRoot.getApplication().getElement()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( "5", @@ -2282,7 +2283,7 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc firstValue.getPane())); assertEquals( decode(inputCoder, residualRootForUnprocessedWindows.getApplication().getElement()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( "5", @@ -2315,37 +2316,37 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc assertThat( mainOutputValues, contains( - WindowedValue.of( + WindowedValues.of( "5:5", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(5)), window1, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( "5:6", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(6)), window1, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( "5:7", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(7)), window1, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( "2:0", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(0)), window1, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( "2:1", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)), window1, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( "2:0", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(0)), window2, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( "2:1", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)), window2, @@ -2407,22 +2408,22 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc assertThat( mainOutputValues, contains( - WindowedValue.of( + WindowedValues.of( "7:0", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(0)), window1, splitValue.getPane()), - WindowedValue.of( + WindowedValues.of( "7:1", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)), window1, splitValue.getPane()), - WindowedValue.of( + WindowedValues.of( "7:2", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(2)), window1, splitValue.getPane()), - WindowedValue.of( + WindowedValues.of( "7:3", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(3)), window1, @@ -2490,7 +2491,7 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc assertEquals( expectedOutputWatermarkMap, residualRoot.getApplication().getOutputWatermarksMap()); assertEquals( - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( "7", @@ -2625,7 +2626,7 @@ public void testProcessElementForTruncateAndSizeRestrictionForwardSplitWhenObser RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(pProto.getComponents()); Coder inputCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( CoderTranslation.fromProto( pProto.getComponents().getCodersOrThrow(inputPCollection.getCoderId()), rehydratedComponents, @@ -2710,14 +2711,14 @@ public void testProcessElementForTruncateAndSizeRestrictionForwardSplitWhenObser assertThat( mainOutputValues, contains( - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("7", KV.of(new OffsetRange(0, 3), GlobalWindow.TIMESTAMP_MIN_VALUE)), 3.0), splitValue.getTimestamp(), window1, splitValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("7", KV.of(new OffsetRange(0, 3), GlobalWindow.TIMESTAMP_MIN_VALUE)), 3.0), @@ -2730,7 +2731,7 @@ public void testProcessElementForTruncateAndSizeRestrictionForwardSplitWhenObser Iterables.getOnlyElement(expectedElementSplit.getPrimaryRoots()); ByteStringOutputStream primaryBytes = new ByteStringOutputStream(); inputCoder.encode( - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("7", KV.of(new OffsetRange(0, 6), GlobalWindow.TIMESTAMP_MIN_VALUE)), 6.0), splitValue.getTimestamp(), @@ -2747,7 +2748,7 @@ public void testProcessElementForTruncateAndSizeRestrictionForwardSplitWhenObser Iterables.getOnlyElement(expectedElementSplit.getResidualRoots()); ByteStringOutputStream residualBytes = new ByteStringOutputStream(); inputCoder.encode( - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("7", KV.of(new OffsetRange(0, 6), GlobalWindow.TIMESTAMP_MIN_VALUE)), 6.0), splitValue.getTimestamp(), @@ -3020,28 +3021,28 @@ public void testProcessElementForWindowedTruncateAndSizeRestriction() throws Exc assertThat( mainOutputValues, contains( - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(0, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 2.0), firstValue.getTimestamp(), window1, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(0, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 2.0), firstValue.getTimestamp(), window2, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(0, 1), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), firstValue.getTimestamp(), window1, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(0, 1), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), @@ -3143,14 +3144,14 @@ public void testProcessElementForWindowedTruncateAndSizeRestriction() throws Exc assertThat( mainOutputValues, contains( - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(0, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 2.0), firstValue.getTimestamp(), ImmutableList.of(window1, window2), firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(0, 1), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), @@ -3333,14 +3334,14 @@ public static class SplitTest { private KV createSplitInWindow( OffsetRange primaryRestriction, OffsetRange residualRestriction, BoundedWindow window) { return KV.of( - WindowedValue.of( + WindowedValues.of( KV.of( currentElement.getValue(), KV.of(primaryRestriction, currentWatermarkEstimatorState)), currentElement.getTimestamp(), window, currentElement.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( currentElement.getValue(), KV.of(residualRestriction, watermarkAndState.getValue())), @@ -3354,7 +3355,7 @@ private KV createSplitAcrossWindows( return KV.of( primaryWindows.isEmpty() ? null - : WindowedValue.of( + : WindowedValues.of( KV.of( currentElement.getValue(), KV.of(currentRestriction, currentWatermarkEstimatorState)), @@ -3363,7 +3364,7 @@ private KV createSplitAcrossWindows( currentElement.getPane()), residualWindows.isEmpty() ? null - : WindowedValue.of( + : WindowedValues.of( KV.of( currentElement.getValue(), KV.of(currentRestriction, currentWatermarkEstimatorState)), @@ -3375,7 +3376,7 @@ private KV createSplitAcrossWindows( private KV createSplitWithSizeInWindow( OffsetRange primaryRestriction, OffsetRange residualRestriction, BoundedWindow window) { return KV.of( - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( currentElement.getValue(), @@ -3384,7 +3385,7 @@ private KV createSplitWithSizeInWindow( currentElement.getTimestamp(), window, currentElement.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( currentElement.getValue(), @@ -3400,7 +3401,7 @@ private KV createSplitWithSizeAcrossWindows( return KV.of( primaryWindows.isEmpty() ? null - : WindowedValue.of( + : WindowedValues.of( KV.of( KV.of( currentElement.getValue(), @@ -3411,7 +3412,7 @@ private KV createSplitWithSizeAcrossWindows( currentElement.getPane()), residualWindows.isEmpty() ? null - : WindowedValue.of( + : WindowedValues.of( KV.of( KV.of( currentElement.getValue(), @@ -3428,7 +3429,7 @@ public void setUp() { window2 = new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)); window3 = new IntervalWindow(Instant.ofEpochMilli(20), Instant.ofEpochMilli(30)); currentElement = - WindowedValue.of( + WindowedValues.of( "a", Instant.ofEpochMilli(57), ImmutableList.of(window1, window2, window3), @@ -4188,13 +4189,13 @@ public void testConstructSplitResultWithInvalidElementSplits() throws Exception FnApiDoFnRunner.constructSplitResult( WindowedSplitResult.forRoots( null, - WindowedValue.valueInGlobalWindow("elementPrimary"), - WindowedValue.valueInGlobalWindow("elementResidual"), + WindowedValues.valueInGlobalWindow("elementPrimary"), + WindowedValues.valueInGlobalWindow("elementResidual"), null), HandlesSplits.SplitResult.of( ImmutableList.of(BundleApplication.getDefaultInstance()), ImmutableList.of(DelayedBundleApplication.getDefaultInstance())), - WindowedValue.getFullCoder(VoidCoder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(VoidCoder.of(), GlobalWindow.Coder.INSTANCE), Instant.now(), null, "ptransformId", @@ -4209,7 +4210,7 @@ private Coder getFullInputCoder( KvCoder.of( KvCoder.of(elementCoder, KvCoder.of(restrictionCoder, watermarkStateCoder)), DoubleCoder.of()); - return WindowedValue.getFullCoder(inputCoder, windowCoder); + return WindowedValues.getFullCoder(inputCoder, windowCoder); } private HandlesSplits.SplitResult getProcessElementSplit(String transformId, String inputId) { diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/MapFnRunnersTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/MapFnRunnersTest.java index ea388b386db3..e881c35ce710 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/MapFnRunnersTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/MapFnRunnersTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.fn.harness; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -36,8 +36,9 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CoderTranslation; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.joda.time.Duration; import org.joda.time.Instant; @@ -153,7 +154,7 @@ public void testFullWindowedValueMappingWithCompressedWindow() throws Exception context .getPCollectionConsumer("inputPC") .accept( - WindowedValue.of( + WindowedValues.of( "abc", new Instant(12), ImmutableSet.of(firstWindow, GlobalWindow.INSTANCE, secondWindow), @@ -162,9 +163,9 @@ public void testFullWindowedValueMappingWithCompressedWindow() throws Exception assertThat( outputConsumer, containsInAnyOrder( - WindowedValue.timestampedValueInGlobalWindow("ABC", new Instant(12)), - WindowedValue.of("ABC", new Instant(12), secondWindow, PaneInfo.NO_FIRING), - WindowedValue.of("ABC", new Instant(12), firstWindow, PaneInfo.NO_FIRING))); + WindowedValues.timestampedValueInGlobalWindow("ABC", new Instant(12)), + WindowedValues.of("ABC", new Instant(12), secondWindow, PaneInfo.NO_FIRING), + WindowedValues.of("ABC", new Instant(12), firstWindow, PaneInfo.NO_FIRING))); } public ThrowingFunction, WindowedValue> diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java index d05e7c72eaf7..7b4387738a4c 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java @@ -52,8 +52,8 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.BundleFinalizer; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Timer; +import org.apache.beam.sdk.values.WindowedValue; import org.joda.time.Instant; /** diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PrecombineGroupingTableTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PrecombineGroupingTableTest.java index 64f160d183d2..384f46bb4527 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PrecombineGroupingTableTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PrecombineGroupingTableTest.java @@ -17,8 +17,8 @@ */ package org.apache.beam.fn.harness; -import static org.apache.beam.sdk.util.WindowedValue.timestampedValueInGlobalWindow; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.timestampedValueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -48,8 +48,8 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ArrayListMultimap; import org.hamcrest.Description; import org.hamcrest.TypeSafeDiagnosingMatcher; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunnerTest.java index 7c52da950882..370fd6e6de2c 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunnerTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.fn.harness; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -58,7 +58,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.util.construction.PipelineTranslation; @@ -68,6 +67,8 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @@ -127,7 +128,7 @@ private org.apache.beam.sdk.util.construction.Timer dynamicTimerInGlobalW private WindowedValue valueInWindows( T value, BoundedWindow window, BoundedWindow... windows) { - return WindowedValue.of( + return WindowedValues.of( value, window.maxTimestamp(), ImmutableList.builder().add(window).add(windows).build(), @@ -353,7 +354,7 @@ public void testProcessElementForWindowedPairWithRestriction() throws Exception assertThat( mainOutputValues, contains( - WindowedValue.of( + WindowedValues.of( KV.of( "5", KV.of( @@ -362,7 +363,7 @@ public void testProcessElementForWindowedPairWithRestriction() throws Exception firstValue.getTimestamp(), window1, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( "5", KV.of( @@ -371,7 +372,7 @@ public void testProcessElementForWindowedPairWithRestriction() throws Exception firstValue.getTimestamp(), window2, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( "2", KV.of( @@ -380,7 +381,7 @@ public void testProcessElementForWindowedPairWithRestriction() throws Exception secondValue.getTimestamp(), window1, secondValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( "2", KV.of( @@ -468,7 +469,7 @@ public void testProcessElementForWindowedPairWithRestrictionWithNonWindowObservi assertThat( mainOutputValues, contains( - WindowedValue.of( + WindowedValues.of( KV.of( "5", KV.of( @@ -477,7 +478,7 @@ public void testProcessElementForWindowedPairWithRestrictionWithNonWindowObservi firstValue.getTimestamp(), ImmutableList.of(window1, window2), firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( "2", KV.of( diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittableSplitAndSizeRestrictionsDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittableSplitAndSizeRestrictionsDoFnRunnerTest.java index 95b9129eb763..a19f65bd5716 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittableSplitAndSizeRestrictionsDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittableSplitAndSizeRestrictionsDoFnRunnerTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.fn.harness; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -55,7 +55,6 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.SlidingWindows; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.util.construction.PipelineTranslation; @@ -65,6 +64,8 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @@ -89,7 +90,7 @@ public class SplittableSplitAndSizeRestrictionsDoFnRunnerTest implements Seriali private WindowedValue valueInWindows( T value, BoundedWindow window, BoundedWindow... windows) { - return WindowedValue.of( + return WindowedValues.of( value, window.maxTimestamp(), ImmutableList.builder().add(window).add(windows).build(), @@ -313,7 +314,7 @@ public void testProcessElementForWindowedSplitAndSizeRestriction() throws Except window1, window2); WindowedValue secondValue = - WindowedValue.of( + WindowedValues.of( KV.of("2", KV.of(new OffsetRange(0, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), firstValue.getTimestamp().plus(Duration.standardSeconds(1)), ImmutableList.of(window1, window2), @@ -327,56 +328,56 @@ public void testProcessElementForWindowedSplitAndSizeRestriction() throws Except assertThat( mainOutputValues, contains( - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(0, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 2.0), firstValue.getTimestamp(), window1, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(2, 5), GlobalWindow.TIMESTAMP_MIN_VALUE)), 3.0), firstValue.getTimestamp(), window1, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(0, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 2.0), firstValue.getTimestamp(), window2, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(2, 5), GlobalWindow.TIMESTAMP_MIN_VALUE)), 3.0), firstValue.getTimestamp(), window2, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(0, 1), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), secondValue.getTimestamp(), window1, secondValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(1, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), secondValue.getTimestamp(), window1, secondValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(0, 1), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), secondValue.getTimestamp(), window2, secondValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(1, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), @@ -470,28 +471,28 @@ public void testProcessElementForWindowedSplitAndSizeRestriction() throws Except assertThat( mainOutputValues, contains( - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(0, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 2.0), firstValue.getTimestamp(), ImmutableList.of(window1, window2), firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(2, 5), GlobalWindow.TIMESTAMP_MIN_VALUE)), 3.0), firstValue.getTimestamp(), ImmutableList.of(window1, window2), firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(0, 1), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), firstValue.getTimestamp(), ImmutableList.of(window1, window2), firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(1, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java index 9776828b4ae2..15f83f2582c7 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java @@ -18,7 +18,7 @@ package org.apache.beam.fn.harness.data; import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.empty; @@ -51,7 +51,8 @@ import org.apache.beam.sdk.fn.test.TestStreams; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ManagedChannel; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.Server; @@ -68,7 +69,7 @@ public class BeamFnDataGrpcClientTest { private static final Coder> CODER = LengthPrefixCoder.of( - WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE)); + WindowedValues.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE)); private static final String INSTRUCTION_ID_A = "12L"; private static final String INSTRUCTION_ID_B = "56L"; private static final String TRANSFORM_ID_A = "34L"; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java index 4f0e1092d90a..f4207d472f84 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.fn.harness.data; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertEquals; @@ -71,10 +71,10 @@ import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterable; import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterator; import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ManagedChannel; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.Server; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java index fc1d089b11ce..5f9c1ecc5a96 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java @@ -40,7 +40,8 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Test; @@ -71,7 +72,7 @@ byte[] encodeByteArray(byte[] b) throws IOException { } WindowedValue globalWindowedValue(T el) { - return WindowedValue.valueInGlobalWindow(el); + return WindowedValues.valueInGlobalWindow(el); } BeamFnApi.InstructionResponse getAllSamples(DataSampler dataSampler) { diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java index 01ddd22b5603..11e44493f377 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java @@ -31,7 +31,8 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Test; @@ -50,11 +51,11 @@ public BeamFnApi.SampledElement encodeInt(Integer i) throws IOException { } public BeamFnApi.SampledElement encodeGlobalWindowedInt(Integer i) throws IOException { - WindowedValue.WindowedValueCoder coder = - WindowedValue.FullWindowedValueCoder.of(VarIntCoder.of(), GlobalWindow.Coder.INSTANCE); + WindowedValues.WindowedValueCoder coder = + WindowedValues.FullWindowedValueCoder.of(VarIntCoder.of(), GlobalWindow.Coder.INSTANCE); ByteArrayOutputStream stream = new ByteArrayOutputStream(); - coder.encode(WindowedValue.valueInGlobalWindow(i), stream); + coder.encode(WindowedValues.valueInGlobalWindow(i), stream); return BeamFnApi.SampledElement.newBuilder() .setElement(ByteString.copyFrom(stream.toByteArray())) .build(); @@ -94,7 +95,7 @@ public void testSamplesFirstN() throws IOException { // Purposely go over maxSamples and sampleEveryN. This helps to increase confidence. for (int i = 0; i < 15; ++i) { - outputSampler.sample(WindowedValue.valueInGlobalWindow(i)); + outputSampler.sample(WindowedValues.valueInGlobalWindow(i)); } // The expected list is only 0..9 inclusive. @@ -109,11 +110,11 @@ public void testSamplesFirstN() throws IOException { @Test public void testWindowedValueSample() throws IOException { - WindowedValue.WindowedValueCoder coder = - WindowedValue.FullWindowedValueCoder.of(VarIntCoder.of(), GlobalWindow.Coder.INSTANCE); + WindowedValues.WindowedValueCoder coder = + WindowedValues.FullWindowedValueCoder.of(VarIntCoder.of(), GlobalWindow.Coder.INSTANCE); OutputSampler outputSampler = new OutputSampler<>(coder, 10, 10, false); - outputSampler.sample(WindowedValue.valueInGlobalWindow(0)); + outputSampler.sample(WindowedValues.valueInGlobalWindow(0)); // The expected list is only 0..9 inclusive. List expected = ImmutableList.of(encodeGlobalWindowedInt(0)); @@ -126,7 +127,7 @@ public void testNonWindowedValueSample() throws IOException { VarIntCoder coder = VarIntCoder.of(); OutputSampler outputSampler = new OutputSampler<>(coder, 10, 10, false); - outputSampler.sample(WindowedValue.valueInGlobalWindow(0)); + outputSampler.sample(WindowedValues.valueInGlobalWindow(0)); // The expected list is only 0..9 inclusive. List expected = ImmutableList.of(encodeInt(0)); @@ -145,7 +146,7 @@ public void testActsLikeCircularBuffer() throws IOException { OutputSampler outputSampler = new OutputSampler<>(coder, 5, 20, false); for (int i = 0; i < 100; ++i) { - outputSampler.sample(WindowedValue.valueInGlobalWindow(i)); + outputSampler.sample(WindowedValues.valueInGlobalWindow(i)); } // The first 10 are always sampled, but with maxSamples = 5, the first ten are downsampled to @@ -173,7 +174,7 @@ public void testCanSampleExceptions() throws IOException { VarIntCoder coder = VarIntCoder.of(); OutputSampler outputSampler = new OutputSampler<>(coder, 5, 20, false); - WindowedValue windowedValue = WindowedValue.valueInGlobalWindow(1); + WindowedValue windowedValue = WindowedValues.valueInGlobalWindow(1); ElementSample elementSample = outputSampler.sample(windowedValue); Exception exception = new RuntimeException("Test exception"); @@ -200,9 +201,9 @@ public void testNoDuplicateExceptions() throws IOException { OutputSampler outputSampler = new OutputSampler<>(coder, 5, 20, false); ElementSample elementSampleA = - outputSampler.sample(WindowedValue.valueInGlobalWindow(1)); + outputSampler.sample(WindowedValues.valueInGlobalWindow(1)); ElementSample elementSampleB = - outputSampler.sample(WindowedValue.valueInGlobalWindow(2)); + outputSampler.sample(WindowedValues.valueInGlobalWindow(2)); Exception exception = new RuntimeException("Test exception"); String ptransformIdA = "ptransformA"; @@ -229,7 +230,7 @@ public void testExceptionOnlySampledIfNonNullProcessBundle() throws IOException VarIntCoder coder = VarIntCoder.of(); OutputSampler outputSampler = new OutputSampler<>(coder, 5, 20, false); - WindowedValue windowedValue = WindowedValue.valueInGlobalWindow(1); + WindowedValue windowedValue = WindowedValues.valueInGlobalWindow(1); ElementSample elementSample = outputSampler.sample(windowedValue); Exception exception = new RuntimeException("Test exception"); @@ -253,11 +254,11 @@ public void testExceptionSamplesAreNotRemoved() throws IOException { VarIntCoder coder = VarIntCoder.of(); OutputSampler outputSampler = new OutputSampler<>(coder, 5, 20, false); - WindowedValue windowedValue = WindowedValue.valueInGlobalWindow(0); + WindowedValue windowedValue = WindowedValues.valueInGlobalWindow(0); ElementSample elementSample = outputSampler.sample(windowedValue); for (int i = 1; i < 100; ++i) { - outputSampler.sample(WindowedValue.valueInGlobalWindow(i)); + outputSampler.sample(WindowedValues.valueInGlobalWindow(i)); } Exception exception = new RuntimeException("Test exception"); @@ -290,8 +291,8 @@ public void testOnlySampleExceptions() throws IOException { VarIntCoder coder = VarIntCoder.of(); OutputSampler outputSampler = new OutputSampler<>(coder, 5, 20, true); - WindowedValue windowedValue = WindowedValue.valueInGlobalWindow(1); - outputSampler.sample(WindowedValue.valueInGlobalWindow(2)); + WindowedValue windowedValue = WindowedValues.valueInGlobalWindow(1); + outputSampler.sample(WindowedValues.valueInGlobalWindow(2)); ElementSample elementSample = outputSampler.sample(windowedValue); Exception exception = new RuntimeException("Test exception"); @@ -333,7 +334,7 @@ public void testConcurrentSamples() throws IOException, InterruptedException { for (int i = 0; i < 1000000; i++) { ElementSample sample = - outputSampler.sample(WindowedValue.valueInGlobalWindow(i)); + outputSampler.sample(WindowedValues.valueInGlobalWindow(i)); outputSampler.exception(sample, new RuntimeException(""), "ptransformId", "pbId"); } @@ -352,7 +353,7 @@ public void testConcurrentSamples() throws IOException, InterruptedException { for (int i = -1000000; i < 0; i++) { ElementSample sample = - outputSampler.sample(WindowedValue.valueInGlobalWindow(i)); + outputSampler.sample(WindowedValues.valueInGlobalWindow(i)); outputSampler.exception(sample, new RuntimeException(""), "ptransformId", "pbId"); } diff --git a/sdks/java/io/components/src/main/java/org/apache/beam/sdk/io/components/deadletterqueue/DLQRouter.java b/sdks/java/io/components/src/main/java/org/apache/beam/sdk/io/components/deadletterqueue/DLQRouter.java index 7cff6cb3e791..6b4485db2167 100644 --- a/sdks/java/io/components/src/main/java/org/apache/beam/sdk/io/components/deadletterqueue/DLQRouter.java +++ b/sdks/java/io/components/src/main/java/org/apache/beam/sdk/io/components/deadletterqueue/DLQRouter.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.components.deadletterqueue; + import java.util.HashMap; import java.util.Map; import org.apache.beam.sdk.transforms.PTransform; diff --git a/sdks/java/io/components/src/main/java/org/apache/beam/sdk/io/components/deadletterqueue/sinks/ThrowingSink.java b/sdks/java/io/components/src/main/java/org/apache/beam/sdk/io/components/deadletterqueue/sinks/ThrowingSink.java index 3e5994e41e3b..9869cd09da3b 100644 --- a/sdks/java/io/components/src/main/java/org/apache/beam/sdk/io/components/deadletterqueue/sinks/ThrowingSink.java +++ b/sdks/java/io/components/src/main/java/org/apache/beam/sdk/io/components/deadletterqueue/sinks/ThrowingSink.java @@ -17,13 +17,13 @@ */ package org.apache.beam.sdk.io.components.deadletterqueue.sinks; + import org.apache.beam.repackaged.core.org.apache.commons.lang3.ObjectUtils.Null; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.Row; import org.checkerframework.checker.nullness.qual.NonNull; public class ThrowingSink extends PTransform<@NonNull PCollection, @NonNull PDone> { diff --git a/sdks/java/io/components/src/test/java/org/apache/beam/sdk/io/components/deadletterqueue/DLQRouterTest.java b/sdks/java/io/components/src/test/java/org/apache/beam/sdk/io/components/deadletterqueue/DLQRouterTest.java index 14e076df1dcc..8d2e5d296f4f 100644 --- a/sdks/java/io/components/src/test/java/org/apache/beam/sdk/io/components/deadletterqueue/DLQRouterTest.java +++ b/sdks/java/io/components/src/test/java/org/apache/beam/sdk/io/components/deadletterqueue/DLQRouterTest.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.components.deadletterqueue; + import org.apache.beam.sdk.io.components.deadletterqueue.sinks.ThrowingSink; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpersTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpersTest.java index 472870805be5..0d36d7bb46d0 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpersTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpersTest.java @@ -43,7 +43,7 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.FluentBackoff; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; import org.joda.time.Duration; @@ -175,7 +175,7 @@ public void testTableRowInfoCoderSerializable() { @Test public void testComplexCoderSerializable() { CoderProperties.coderSerializable( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( ShardedKeyCoder.of(StringUtf8Coder.of()), TableRowInfoCoder.of(TableRowJsonCoder.of())), diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriterManager.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriterManager.java index bbde6c9e864e..142170acc67b 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriterManager.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriterManager.java @@ -33,8 +33,8 @@ import java.util.concurrent.TimeUnit; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.util.Preconditions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache; diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java index 6a61aafbe8b9..42de901587b6 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java @@ -24,10 +24,11 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.ShardedKey; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.iceberg.catalog.Catalog; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; @@ -98,7 +99,7 @@ public void processElement( String tableIdentifier = element.getKey().getKey(); IcebergDestination destination = dynamicDestinations.instantiateDestination(tableIdentifier); WindowedValue windowedDestination = - WindowedValue.of(destination, window.maxTimestamp(), window, pane); + WindowedValues.of(destination, window.maxTimestamp(), window, pane); RecordWriterManager writer; try (RecordWriterManager openWriter = new RecordWriterManager(getCatalog(), filePrefix, maxFileSize, Integer.MAX_VALUE)) { diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java index 80bf962283f0..e76d18ae7106 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java @@ -31,7 +31,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.ShardedKey; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; @@ -41,6 +40,8 @@ import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -233,7 +234,7 @@ public void processElement( Row data = element.getValue(); IcebergDestination destination = dynamicDestinations.instantiateDestination(dest); WindowedValue windowedDestination = - WindowedValue.of(destination, window.maxTimestamp(), window, pane); + WindowedValues.of(destination, window.maxTimestamp(), window, pane); // Attempt to write record. If the writer is saturated and cannot accept // the record, spill it over to WriteGroupedRowsToFiles diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/RecordWriterManagerTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/RecordWriterManagerTest.java index 5d93f10a43d1..1fef79e81240 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/RecordWriterManagerTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/RecordWriterManagerTest.java @@ -43,8 +43,9 @@ import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.conf.Configuration; @@ -117,7 +118,7 @@ private WindowedValue getWindowedDestination( .setFileFormat(FileFormat.PARQUET) .setTableIdentifier(tableIdentifier) .build(); - return WindowedValue.of( + return WindowedValues.of( icebergDestination, GlobalWindow.TIMESTAMP_MAX_VALUE, GlobalWindow.INSTANCE, @@ -745,7 +746,7 @@ public void testColumnSpecificMetricsCollection() throws IOException { .setFileFormat(FileFormat.PARQUET) .build(); WindowedValue singleDestination = - WindowedValue.valueInGlobalWindow(destination); + WindowedValues.valueInGlobalWindow(destination); RecordWriterManager writerManager = new RecordWriterManager(catalog, "test_file_name", 1000, 3); Row row1 = Row.withSchema(BEAM_SCHEMA).addValues(1, "aaa", true).build(); @@ -807,7 +808,7 @@ public void testDefaultMetrics() throws IOException { .setFileFormat(FileFormat.PARQUET) .build(); WindowedValue singleDestination = - WindowedValue.valueInGlobalWindow(destination); + WindowedValues.valueInGlobalWindow(destination); RecordWriterManager writerManager = new RecordWriterManager(catalog, "test_file_name", 1000, 3); Row row1 = Row.withSchema(BEAM_SCHEMA).addValues(1, "aaa", true).build(); From 479f0d680422a8bd85adc29f2919abb5ed7f519e Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 5 Jun 2025 10:24:32 -0400 Subject: [PATCH 2/3] Run integration tests for moving WindowedValue and making public --- .github/trigger_files/IO_Iceberg_Integration_Tests.json | 3 ++- .../trigger_files/IO_Iceberg_Integration_Tests_Dataflow.json | 3 ++- .github/trigger_files/beam_PostCommit_Java_DataflowV1.json | 3 ++- .github/trigger_files/beam_PostCommit_Java_DataflowV2.json | 3 ++- .../beam_PostCommit_Java_ValidatesRunner_Dataflow.json | 3 ++- ...eam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.json | 3 ++- .../beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.json | 3 ++- .../beam_PostCommit_Java_ValidatesRunner_Direct.json | 3 ++- .../beam_PostCommit_Java_ValidatesRunner_Flink.json | 3 ++- .../beam_PostCommit_Java_ValidatesRunner_Samza.json | 3 ++- .../beam_PostCommit_Java_ValidatesRunner_Spark.json | 3 ++- ...stCommit_Java_ValidatesRunner_SparkStructuredStreaming.json | 3 ++- .../beam_PostCommit_Java_ValidatesRunner_ULR.json | 3 ++- 13 files changed, 26 insertions(+), 13 deletions(-) diff --git a/.github/trigger_files/IO_Iceberg_Integration_Tests.json b/.github/trigger_files/IO_Iceberg_Integration_Tests.json index 37dd25bf9029..98be2d60cbf9 100644 --- a/.github/trigger_files/IO_Iceberg_Integration_Tests.json +++ b/.github/trigger_files/IO_Iceberg_Integration_Tests.json @@ -1,4 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 3 + "modification": 3, + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/IO_Iceberg_Integration_Tests_Dataflow.json b/.github/trigger_files/IO_Iceberg_Integration_Tests_Dataflow.json index 3a009261f4f9..316d6e41388d 100644 --- a/.github/trigger_files/IO_Iceberg_Integration_Tests_Dataflow.json +++ b/.github/trigger_files/IO_Iceberg_Integration_Tests_Dataflow.json @@ -1,4 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 2 + "modification": 2, + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_DataflowV1.json b/.github/trigger_files/beam_PostCommit_Java_DataflowV1.json index 1efc8e9e4405..156f95fbeb1c 100644 --- a/.github/trigger_files/beam_PostCommit_Java_DataflowV1.json +++ b/.github/trigger_files/beam_PostCommit_Java_DataflowV1.json @@ -1,4 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 1 + "modification": 1, + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_DataflowV2.json b/.github/trigger_files/beam_PostCommit_Java_DataflowV2.json index bbdc3a3910ef..085929db9af8 100644 --- a/.github/trigger_files/beam_PostCommit_Java_DataflowV2.json +++ b/.github/trigger_files/beam_PostCommit_Java_DataflowV2.json @@ -1,4 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 3 + "modification": 3, + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json index 8aad4bae0703..85482285d1ae 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json @@ -1,5 +1,6 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", "modification": 2, - "https://github.com/apache/beam/pull/34294": "noting that PR #34294 should run this test" + "https://github.com/apache/beam/pull/34294": "noting that PR #34294 should run this test", + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.json index 96e098eb7f97..c695f7cb67b7 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.json @@ -2,5 +2,6 @@ "comment": "Modify this file in a trivial way to cause this test suite to run", "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test", "https://github.com/apache/beam/pull/31268": "noting that PR #31268 should run this test", - "https://github.com/apache/beam/pull/31490": "noting that PR #31490 should run this test" + "https://github.com/apache/beam/pull/31490": "noting that PR #31490 should run this test", + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.json index 96e098eb7f97..c695f7cb67b7 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.json @@ -2,5 +2,6 @@ "comment": "Modify this file in a trivial way to cause this test suite to run", "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test", "https://github.com/apache/beam/pull/31268": "noting that PR #31268 should run this test", - "https://github.com/apache/beam/pull/31490": "noting that PR #31490 should run this test" + "https://github.com/apache/beam/pull/31490": "noting that PR #31490 should run this test", + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Direct.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Direct.json index 38ae94aee2fa..27b4484dbd23 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Direct.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Direct.json @@ -1,4 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "https://github.com/apache/beam/pull/31761": "noting that PR #31761 should run this test" + "https://github.com/apache/beam/pull/31761": "noting that PR #31761 should run this test", + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink.json index e375c3e6c505..6572defd2538 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink.json @@ -1,5 +1,6 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", "https://github.com/apache/beam/pull/32440": "testing datastream optimizations", - "runFor": "#33606" + "runFor": "#33606", + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Samza.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Samza.json index 1d4a0589a276..f838c61661aa 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Samza.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Samza.json @@ -1,5 +1,6 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test", - "https://github.com/apache/beam/pull/31270": "re-add specialized Samza translation of Redistribute" + "https://github.com/apache/beam/pull/31270": "re-add specialized Samza translation of Redistribute", + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark.json index 64f9b2e34efa..9649514a5df5 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark.json @@ -8,5 +8,6 @@ "https://github.com/apache/beam/pull/34123": "noting that PR #34123 should run this test", "https://github.com/apache/beam/pull/34080": "noting that PR #34080 should run this test", "https://github.com/apache/beam/pull/34155": "noting that PR #34155 should run this test", - "https://github.com/apache/beam/pull/34560": "noting that PR #34560 should run this test" + "https://github.com/apache/beam/pull/34560": "noting that PR #34560 should run this test", + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.json index fb9656ec5f68..77f63217b86d 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.json @@ -6,5 +6,6 @@ "https://github.com/apache/beam/pull/33267": "noting that PR #33267 should run this test", "https://github.com/apache/beam/pull/34123": "noting that PR #34123 should run this test", "https://github.com/apache/beam/pull/34080": "noting that PR #34080 should run this test", - "https://github.com/apache/beam/pull/34155": "noting that PR #34155 should run this test" + "https://github.com/apache/beam/pull/34155": "noting that PR #34155 should run this test", + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_ULR.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_ULR.json index b970762c8397..26d472693709 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_ULR.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_ULR.json @@ -1,4 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test" + "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test", + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } From deff583cf6eb41cc162c57b862a245df77f2a216 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Thu, 5 Jun 2025 14:44:10 -0400 Subject: [PATCH 3/3] Make SDK harness change effective on Iceberg Dataflow test --- sdks/java/io/iceberg/build.gradle | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index 8d5cef919949..4d79fb061d05 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -95,6 +95,7 @@ dependencies { testRuntimeOnly library.java.slf4j_jdk14 testImplementation project(path: ":runners:direct-java", configuration: "shadow") testRuntimeOnly project(path: ":runners:google-cloud-dataflow-java") + testRuntimeOnly project(path: ":sdks:java:harness") hadoopVersions.each {kv -> "hadoopVersion$kv.key" "org.apache.hadoop:hadoop-client:$kv.value" "hadoopVersion$kv.key" "org.apache.hadoop:hadoop-minicluster:$kv.value" @@ -165,7 +166,7 @@ task dataflowIntegrationTest(type: Test) { "--project=${gcpProject}", "--tempLocation=${gcpTempLocation}", "--tempRoot=${gcpTempLocation}", - "--experiments=use_runner_v2" + "--experiments=use_runner_v2,use_staged_dataflow_worker_jar" ] if (project.hasProperty('enableManagedTransforms')) { args.add("--experiments=enable_managed_transforms")