From bc30ecc75a70f9e5a2a38eb9a3912481258c1478 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Thu, 22 Aug 2024 07:33:37 -0700 Subject: [PATCH 01/33] Initial changes to support processing global sequences. --- sdks/java/extensions/ordered/build.gradle | 1 + .../ordered/GlobalSequenceTracker.java | 165 +++++ .../ordered/OrderedEventProcessor.java | 149 ++-- .../ordered/OrderedProcessingHandler.java | 10 + .../sdk/extensions/ordered/SequenceType.java | 6 + ...deredEventProcessorGlobalSequenceTest.java | 671 ++++++++++++++++++ ...deredEventProcessorPerKeySequenceTest.java | 669 +++++++++++++++++ .../ordered/OrderedEventProcessorTest.java | 666 +---------------- 8 files changed, 1644 insertions(+), 693 deletions(-) create mode 100644 sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java create mode 100644 sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequenceType.java create mode 100644 sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java create mode 100644 sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java diff --git a/sdks/java/extensions/ordered/build.gradle b/sdks/java/extensions/ordered/build.gradle index 10c9785b9eed..dbf88483c50d 100644 --- a/sdks/java/extensions/ordered/build.gradle +++ b/sdks/java/extensions/ordered/build.gradle @@ -28,6 +28,7 @@ dependencies { implementation library.java.vendored_guava_32_1_2_jre testImplementation library.java.junit testImplementation library.java.hamcrest + testImplementation library.java.slf4j_simple testImplementation project(path: ':sdks:java:core') testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") } \ No newline at end of file diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java new file mode 100644 index 000000000000..342f09755028 --- /dev/null +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java @@ -0,0 +1,165 @@ +package org.apache.beam.sdk.extensions.ordered; + +import com.google.auto.value.AutoValue; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.function.BiFunction; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.extensions.ordered.GlobalSequenceTracker.SequenceAndTimestamp; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.Combine.CombineFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.windowing.AfterFirst; +import org.apache.beam.sdk.transforms.windowing.AfterPane; +import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.Repeatedly; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Range; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.RangeMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.TreeRangeMap; +import org.checkerframework.checker.initialization.qual.Initialized; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.checkerframework.checker.nullness.qual.UnknownKeyFor; +import org.joda.time.Duration; +import org.joda.time.Instant; + +class GlobalSequenceTracker extends + PTransform, PCollectionView> { + + @Override + public PCollectionView expand(PCollection input) { + return + input + .apply("Setup Triggering", Window.into(new GlobalWindows()) + .accumulatingFiredPanes() + .triggering( + Repeatedly.forever(AfterFirst.of( + AfterPane.elementCountAtLeast(1), + AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(Duration.standardSeconds(5)))))) + .apply("Create Side Input", + Combine.globally(new GlobalSequenceCombiner()).asSingletonView()); + } + + @DefaultSchema(AutoValueSchema.class) + @AutoValue + static abstract class SequenceAndTimestamp { + + public abstract long getSequence(); + + public abstract Instant getTimestamp(); + + public static SequenceAndTimestamp create(long sequence, Instant timestamp) { + return new AutoValue_GlobalSequenceTracker_SequenceAndTimestamp(sequence, timestamp); + } + } + + static class GlobalSequenceCombiner extends + CombineFn, SequenceAndTimestamp> { + + public static final BiFunction<@NonNull Instant, @Nullable Instant, @Nullable Instant> OLDEST_TIMESTAMP_SELECTOR = (instant1, instant2) -> { + if (instant2 == null) { + return instant1; + } + @NonNull Instant nonNullableSecondValue = instant2; + return instant1.isAfter(nonNullableSecondValue) ? instant1 : nonNullableSecondValue; + }; + + + @Override + public RangeMap createAccumulator() { + return TreeRangeMap.create(); + } + + @Override + public RangeMap addInput(RangeMap accum, + SequenceAndTimestamp event) { + accum.merge(Range.singleton(event.getSequence()), event.getTimestamp(), + OLDEST_TIMESTAMP_SELECTOR); + return accum; + } + + @Override + public RangeMap mergeAccumulators( + Iterable> accumulators) { + RangeMap newAccum = createAccumulator(); + for (RangeMap accum : accumulators) { + for (Map.Entry, Instant> entry : accum.asMapOfRanges().entrySet()) { + newAccum.merge(entry.getKey(), entry.getValue(), OLDEST_TIMESTAMP_SELECTOR); + } + } + return newAccum; + } + + @Override + public SequenceAndTimestamp extractOutput(RangeMap accum) { + SequenceAndTimestamp output = SequenceAndTimestamp.create(Long.MIN_VALUE, Instant.EPOCH); + Iterator, Instant>> iter = accum.asMapOfRanges().entrySet().iterator(); + Map.Entry, Instant> prevEntry = null; + while (iter.hasNext()) { + Map.Entry, Instant> entry = iter.next(); + if (prevEntry != null + && entry.getKey().lowerEndpoint() > prevEntry.getKey().upperEndpoint() + 1) { + // We have a hole! Break out. + break; + } + output = SequenceAndTimestamp.create(entry.getKey().upperEndpoint(), entry.getValue()); + prevEntry = entry; + } + return output; + } + + + static class AccumulatorCoder extends Coder> { + + @Override + public void encode(RangeMap value, + @UnknownKeyFor @NonNull @Initialized OutputStream outStream) + throws @UnknownKeyFor@NonNull@Initialized CoderException, @UnknownKeyFor@NonNull@Initialized IOException { + + } + + @Override + public RangeMap decode( + @UnknownKeyFor @NonNull @Initialized InputStream inStream) + throws @UnknownKeyFor@NonNull@Initialized CoderException, @UnknownKeyFor@NonNull@Initialized IOException { + return TreeRangeMap.create(); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized List> getCoderArguments() { + return new ArrayList>>(); + } + + @Override + public void verifyDeterministic() + throws @UnknownKeyFor@NonNull@Initialized NonDeterministicException { + + } + } + + @Override + public @UnknownKeyFor @NonNull @Initialized Coder> getAccumulatorCoder( + @UnknownKeyFor @NonNull @Initialized CoderRegistry registry, + @UnknownKeyFor @NonNull @Initialized Coder inputCoder) + throws @UnknownKeyFor@NonNull@Initialized CannotProvideCoderException { + return new AccumulatorCoder(); + } + } +} diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java index 935647c0e7e5..adbc3987f9fe 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java @@ -27,6 +27,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.extensions.ordered.GlobalSequenceTracker.SequenceAndTimestamp; import org.apache.beam.sdk.extensions.ordered.ProcessingState.ProcessingStateCoder; import org.apache.beam.sdk.extensions.ordered.UnprocessedEvent.Reason; import org.apache.beam.sdk.extensions.ordered.UnprocessedEvent.UnprocessedEventCoder; @@ -48,6 +49,7 @@ 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.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; @@ -69,18 +71,20 @@ @AutoValue @SuppressWarnings({"nullness", "TypeNameShadowing"}) public abstract class OrderedEventProcessor< - EventT, EventKeyT, ResultT, StateT extends MutableState> + EventT, EventKeyT, ResultT, StateT extends MutableState> extends PTransform< - PCollection>>, - OrderedEventProcessorResult> { + PCollection>>, + OrderedEventProcessorResult> { + + public static final String GLOBAL_SEQUENCE_TRACKER = "global_sequence_tracker"; public static < - EventTypeT, - EventKeyTypeT, - ResultTypeT, - StateTypeT extends MutableState> - OrderedEventProcessor create( - OrderedProcessingHandler handler) { + EventTypeT, + EventKeyTypeT, + ResultTypeT, + StateTypeT extends MutableState> + OrderedEventProcessor create( + OrderedProcessingHandler handler) { return new AutoValue_OrderedEventProcessor<>(handler); } @@ -91,12 +95,15 @@ OrderedEventProcessor create public OrderedEventProcessorResult expand( PCollection>> input) { final TupleTag> mainOutput = - new TupleTag>("mainOutput") {}; + new TupleTag>("mainOutput") { + }; final TupleTag> statusOutput = - new TupleTag>("status") {}; + new TupleTag>("status") { + }; final TupleTag>>> unprocessedEventOutput = - new TupleTag>>>("unprocessed-events") {}; + new TupleTag>>>("unprocessed-events") { + }; OrderedProcessingHandler handler = getHandler(); Pipeline pipeline = input.getPipeline(); @@ -129,23 +136,71 @@ public OrderedEventProcessorResult expand( throw new RuntimeException("Unable to get result coder", e); } - PCollectionTuple processingResult = - input.apply( - ParDo.of( - new OrderedProcessorDoFn<>( - handler.getEventExaminer(), - eventCoder, - stateCoder, - keyCoder, + PCollectionTuple processingResult; + + switch (handler.getSequenceType()) { + case GLOBAL: + DoFn>, SequenceAndTimestamp> fn = new DoFn>, SequenceAndTimestamp>() { + @ProcessElement + public void convert(@Element KV> element, + @Timestamp Instant timestamp, + OutputReceiver outputReceiver + ) { + outputReceiver.output( + SequenceAndTimestamp.create(element.getValue().getKey(), timestamp)); + } + }; + + final PCollectionView latestContinousSequence = + input + .apply("Convert to SequenceAndTimestamp", ParDo.of(fn)) + .apply("GlobalSequenceTracker", new GlobalSequenceTracker()); + processingResult = + input.apply( + ParDo.of( + new OrderedProcessorDoFn<>( + handler.getEventExaminer(), + eventCoder, + stateCoder, + keyCoder, + mainOutput, + statusOutput, + handler.getStatusUpdateFrequency(), + unprocessedEventOutput, + handler.isProduceStatusUpdateOnEveryEvent(), + handler.getMaxOutputElementsPerBundle(), + latestContinousSequence)) + .withOutputTags( + mainOutput, + TupleTagList.of(Arrays.asList(statusOutput, unprocessedEventOutput))) + .withSideInput(GLOBAL_SEQUENCE_TRACKER, latestContinousSequence) +); + break; + + case PER_KEY: + processingResult = + input.apply( + ParDo.of( + new OrderedProcessorDoFn<>( + handler.getEventExaminer(), + eventCoder, + stateCoder, + keyCoder, + mainOutput, + statusOutput, + handler.getStatusUpdateFrequency(), + unprocessedEventOutput, + handler.isProduceStatusUpdateOnEveryEvent(), + handler.getMaxOutputElementsPerBundle(), + null)) + .withOutputTags( mainOutput, - statusOutput, - handler.getStatusUpdateFrequency(), - unprocessedEventOutput, - handler.isProduceStatusUpdateOnEveryEvent(), - handler.getMaxOutputElementsPerBundle())) - .withOutputTags( - mainOutput, - TupleTagList.of(Arrays.asList(statusOutput, unprocessedEventOutput)))); + TupleTagList.of(Arrays.asList(statusOutput, unprocessedEventOutput)))); + break; + + default: + throw new IllegalStateException("Unprocessed sequence type: " + handler.getSequenceType()); + } KvCoder mainOutputCoder = KvCoder.of(keyCoder, resultCoder); KvCoder processingStatusCoder = @@ -187,10 +242,10 @@ private static Coder getOrderedProcessingStatusCoder(Pi * @param */ static class OrderedProcessorDoFn< - EventTypeT, - EventKeyTypeT, - ResultTypeT, - StateTypeT extends MutableState> + EventTypeT, + EventKeyTypeT, + ResultTypeT, + StateTypeT extends MutableState> extends DoFn>, KV> { private static final Logger LOG = LoggerFactory.getLogger(OrderedProcessorDoFn.class); @@ -237,6 +292,8 @@ static class OrderedProcessorDoFn< private final long maxNumberOfResultsToProduce; + private final PCollectionView globalSequenceView; + private Long numberOfResultsBeforeBundleStart; /** @@ -252,6 +309,7 @@ static class OrderedProcessorDoFn< * @param unprocessedEventTupleTag * @param produceStatusUpdateOnEveryEvent * @param maxNumberOfResultsToProduce + * @param globalSequenceView */ OrderedProcessorDoFn( EventExaminer eventExaminer, @@ -264,11 +322,13 @@ static class OrderedProcessorDoFn< TupleTag>>> unprocessedEventTupleTag, boolean produceStatusUpdateOnEveryEvent, - long maxNumberOfResultsToProduce) { + long maxNumberOfResultsToProduce, + PCollectionView globalSequenceView) { this.eventExaminer = eventExaminer; this.bufferedEventsSpec = StateSpecs.orderedList(eventCoder); this.mutableStateSpec = StateSpecs.value(stateCoder); this.processingStateSpec = StateSpecs.value(ProcessingStateCoder.of(keyCoder)); + this.globalSequenceView = globalSequenceView; this.windowClosedSpec = StateSpecs.value(BooleanCoder.of()); this.mainOutputTupleTag = mainOutputTupleTag; this.statusTupleTag = statusTupleTag; @@ -293,13 +353,19 @@ public void onBundleFinish() { public void processElement( @StateId(BUFFERED_EVENTS) OrderedListState bufferedEventsState, @AlwaysFetched @StateId(PROCESSING_STATE) - ValueState> processingStateState, + ValueState> processingStateState, @StateId(MUTABLE_STATE) ValueState mutableStateState, @TimerId(STATUS_EMISSION_TIMER) Timer statusEmissionTimer, @TimerId(LARGE_BATCH_EMISSION_TIMER) Timer largeBatchEmissionTimer, @Element KV> eventAndSequence, MultiOutputReceiver outputReceiver, - BoundedWindow window) { + BoundedWindow window, + ProcessContext context) { + + if (globalSequenceView != null) { + SequenceAndTimestamp latestGlobalSequence = context.sideInput(globalSequenceView); + LOG.info("Latest sequence: " + latestGlobalSequence.getSequence()); + } EventKeyTypeT key = eventAndSequence.getKey(); long sequence = eventAndSequence.getValue().getKey(); @@ -492,7 +558,9 @@ private StateTypeT processNewEvent( return null; } - /** Process buffered events. */ + /** + * Process buffered events. + */ private void processBufferedEvents( ProcessingState processingState, StateTypeT state, @@ -555,6 +623,9 @@ private void processBufferedEvents( break; } + // Remove this record also + endClearRange = Instant.ofEpochMilli(eventSequence + 1); + try { state.mutate(bufferedEvent); } catch (Exception e) { @@ -575,8 +646,6 @@ private void processBufferedEvents( processingState.resultProduced(); } processingState.processedBufferedEvent(eventSequence); - // Remove this record also - endClearRange = Instant.ofEpochMilli(eventSequence + 1); } bufferedEventsState.clearRange(startRange, endClearRange); @@ -605,7 +674,7 @@ public void onBatchEmission( OnTimerContext context, @StateId(BUFFERED_EVENTS) OrderedListState bufferedEventsState, @AlwaysFetched @StateId(PROCESSING_STATE) - ValueState> processingStatusState, + ValueState> processingStatusState, @AlwaysFetched @StateId(MUTABLE_STATE) ValueState currentStateState, @TimerId(LARGE_BATCH_EMISSION_TIMER) Timer largeBatchEmissionTimer, MultiOutputReceiver outputReceiver) { @@ -646,7 +715,7 @@ public void onStatusEmission( @TimerId(STATUS_EMISSION_TIMER) Timer statusEmissionTimer, @StateId(WINDOW_CLOSED) ValueState windowClosedState, @StateId(PROCESSING_STATE) - ValueState> processingStateState) { + ValueState> processingStateState) { ProcessingState currentState = processingStateState.read(); if (currentState == null) { diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java index 444fdb118091..8f8309d340b6 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java @@ -53,6 +53,8 @@ public abstract class OrderedProcessingHandler< Duration.standardSeconds(DEFAULT_STATUS_UPDATE_FREQUENCY_SECONDS); private boolean produceStatusUpdateOnEveryEvent = DEFAULT_PRODUCE_STATUS_UPDATE_ON_EVERY_EVENT; + private SequenceType sequenceType = SequenceType.PER_KEY; + /** * Provide concrete classes which will be used by the ordered processing transform. * @@ -217,4 +219,12 @@ public int getMaxOutputElementsPerBundle() { public void setMaxOutputElementsPerBundle(int maxOutputElementsPerBundle) { this.maxOutputElementsPerBundle = maxOutputElementsPerBundle; } + + public SequenceType getSequenceType() { + return sequenceType; + } + + public void setSequenceType(SequenceType sequenceType) { + this.sequenceType = sequenceType; + } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequenceType.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequenceType.java new file mode 100644 index 000000000000..b1de8020c741 --- /dev/null +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequenceType.java @@ -0,0 +1,6 @@ +package org.apache.beam.sdk.extensions.ordered; + +public enum SequenceType { + GLOBAL, + PER_KEY +} diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java new file mode 100644 index 000000000000..9a0d3fc8376a --- /dev/null +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java @@ -0,0 +1,671 @@ +package org.apache.beam.sdk.extensions.ordered; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.extensions.ordered.UnprocessedEvent.Reason; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestStream; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Test; + +public class OrderedEventProcessorGlobalSequenceTest extends OrderedEventProcessorTest { + + @org.junit.Test + public void testPerfectOrderingProcessing() throws CannotProvideCoderException { + Event[] events = { + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + Event.create(2, "id-1", "c"), + Event.create(3, "id-1", "d"), + Event.create(4, "id-2", "a"), + Event.create(5, "id-2", "b") + }; + + Collection> expectedStatuses = new ArrayList<>(); + expectedStatuses.add( + KV.of( + "id-1", + OrderedProcessingStatus.create( + 3L, + 0, + null, + null, + 4, + Arrays.stream(events).filter(e -> e.getKey().equals("id-1")).count(), + 0, + false))); + expectedStatuses.add( + KV.of( + "id-2", + OrderedProcessingStatus.create( + 1L, + 0, + null, + null, + 2, + Arrays.stream(events).filter(e -> e.getKey().equals("id-2")).count(), + 0, + false))); + + Collection> expectedOutput = new ArrayList<>(); + expectedOutput.add(KV.of("id-1", "a")); + expectedOutput.add(KV.of("id-1", "ab")); + expectedOutput.add(KV.of("id-1", "abc")); + expectedOutput.add(KV.of("id-1", "abcd")); + expectedOutput.add(KV.of("id-2", "a")); + expectedOutput.add(KV.of("id-2", "ab")); + + testGlobalSequenceProcessing( + events, + expectedStatuses, + expectedOutput, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT, + DONT_PRODUCE_STATUS_ON_EVERY_EVENT); + } + + @Test + public void testOutOfSequenceProcessing() throws CannotProvideCoderException { + Event[] events = { + Event.create(2, "id-1", "c"), + Event.create(1, "id-1", "b"), + Event.create(0, "id-1", "a"), + Event.create(3, "id-1", "d"), + Event.create(1, "id-2", "b"), + Event.create(2, "id-2", "c"), + Event.create(4, "id-2", "e"), + Event.create(0, "id-2", "a"), + Event.create(3, "id-2", "d") + }; + + Collection> expectedStatuses = new ArrayList<>(); + expectedStatuses.add( + KV.of( + "id-1", + OrderedProcessingStatus.create( + 3L, + 0, + null, + null, + 4, + Arrays.stream(events).filter(e -> e.getKey().equals("id-1")).count(), + 0, + false))); + expectedStatuses.add( + KV.of( + "id-2", + OrderedProcessingStatus.create( + 4L, + 0, + null, + null, + 5, + Arrays.stream(events).filter(e -> e.getKey().equals("id-2")).count(), + 0, + false))); + + Collection> expectedOutput = new ArrayList<>(); + expectedOutput.add(KV.of("id-1", "a")); + expectedOutput.add(KV.of("id-1", "ab")); + expectedOutput.add(KV.of("id-1", "abc")); + expectedOutput.add(KV.of("id-1", "abcd")); + expectedOutput.add(KV.of("id-2", "a")); + expectedOutput.add(KV.of("id-2", "ab")); + expectedOutput.add(KV.of("id-2", "abc")); + expectedOutput.add(KV.of("id-2", "abcd")); + expectedOutput.add(KV.of("id-2", "abcde")); + + testGlobalSequenceProcessing( + events, + expectedStatuses, + expectedOutput, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT, + DONT_PRODUCE_STATUS_ON_EVERY_EVENT); + } + + @Test + public void testUnfinishedProcessing() throws CannotProvideCoderException { + Event[] events = { + Event.create(2, "id-1", "c"), + // Excluded Event.create(1, "id-1", "b"), + Event.create(0, "id-1", "a"), + Event.create(3, "id-1", "d"), + Event.create(0, "id-2", "a"), + Event.create(1, "id-2", "b"), + }; + + Collection> expectedStatuses = new ArrayList<>(); + expectedStatuses.add( + KV.of("id-1", OrderedProcessingStatus.create(0L, 2, 2L, 3L, 3, 1L, 0, false))); + expectedStatuses.add( + KV.of("id-2", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2L, 0, false))); + + Collection> expectedOutput = new ArrayList<>(); + expectedOutput.add(KV.of("id-1", "a")); + expectedOutput.add(KV.of("id-2", "a")); + expectedOutput.add(KV.of("id-2", "ab")); + + testGlobalSequenceProcessing(events, expectedStatuses, expectedOutput, 1, 0, 1000, false); + } + + @Test + public void testHandlingOfDuplicateSequences() throws CannotProvideCoderException { + Event[] events = { + Event.create(3, "id-1", "d"), + Event.create(2, "id-1", "c"), + // Duplicates to be buffered + Event.create(3, "id-1", "d"), + Event.create(3, "id-1", "d"), + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + + // Duplicates after the events are processed + Event.create(1, "id-1", "b"), + Event.create(3, "id-1", "d"), + }; + int resultCount = 4; + int duplicateCount = 4; + + Collection> expectedStatuses = new ArrayList<>(); + expectedStatuses.add( + KV.of( + "id-1", + OrderedProcessingStatus.create( + 3L, 0, null, null, events.length, resultCount, duplicateCount, false))); + + Collection> expectedOutput = new ArrayList<>(); + expectedOutput.add(KV.of("id-1", "a")); + expectedOutput.add(KV.of("id-1", "ab")); + expectedOutput.add(KV.of("id-1", "abc")); + expectedOutput.add(KV.of("id-1", "abcd")); + + Collection>>> duplicates = new ArrayList<>(); + duplicates.add(KV.of("id-1", KV.of(3L, UnprocessedEvent.create("d", Reason.duplicate)))); + duplicates.add(KV.of("id-1", KV.of(3L, UnprocessedEvent.create("d", Reason.duplicate)))); + duplicates.add(KV.of("id-1", KV.of(1L, UnprocessedEvent.create("b", Reason.duplicate)))); + duplicates.add(KV.of("id-1", KV.of(3L, UnprocessedEvent.create("d", Reason.duplicate)))); + + testGlobalSequenceProcessing( + events, + expectedStatuses, + expectedOutput, + duplicates, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT, + DONT_PRODUCE_STATUS_ON_EVERY_EVENT); + } + + @Test + public void testHandlingOfCheckedExceptions() throws CannotProvideCoderException { + Event[] events = { + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + Event.create(2, "id-1", StringBuilderState.BAD_VALUE), + Event.create(3, "id-1", "c"), + }; + + Collection> expectedStatuses = new ArrayList<>(); + expectedStatuses.add( + KV.of("id-1", OrderedProcessingStatus.create(1L, 1, 3L, 3L, events.length, 2, 0, false))); + + Collection> expectedOutput = new ArrayList<>(); + expectedOutput.add(KV.of("id-1", "a")); + expectedOutput.add(KV.of("id-1", "ab")); + + Collection>>> failedEvents = new ArrayList<>(); + failedEvents.add( + KV.of( + "id-1", + KV.of( + 2L, + UnprocessedEvent.create(StringBuilderState.BAD_VALUE, Reason.exception_thrown)))); + + testGlobalSequenceProcessing( + events, + expectedStatuses, + expectedOutput, + failedEvents, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT, + DONT_PRODUCE_STATUS_ON_EVERY_EVENT); + } + + @Test + public void testProcessingWithEveryOtherResultEmission() throws CannotProvideCoderException { + Event[] events = { + Event.create(2, "id-1", "c"), + Event.create(1, "id-1", "b"), + Event.create(0, "id-1", "a"), + Event.create(3, "id-1", "d"), + Event.create(0, "id-2", "a"), + Event.create(1, "id-2", "b"), + }; + + Collection> expectedStatuses = new ArrayList<>(); + expectedStatuses.add( + KV.of("id-1", OrderedProcessingStatus.create(3L, 0, null, null, 4, 2L, 0, false))); + expectedStatuses.add( + KV.of("id-2", OrderedProcessingStatus.create(1L, 0, null, null, 2, 1L, 0, false))); + + Collection> expectedOutput = new ArrayList<>(); + expectedOutput.add(KV.of("id-1", "a")); + // Skipped KV.of("id-1", "ab"), + expectedOutput.add(KV.of("id-1", "abc")); + // Skipped KV.of("id-1", "abcd"), + expectedOutput.add(KV.of("id-2", "a")); + // Skipped KV.of("id-2", "ab") + testGlobalSequenceProcessing( + events, + expectedStatuses, + expectedOutput, + EMISSION_FREQUENCY_ON_EVERY_OTHER_EVENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT, + DONT_PRODUCE_STATUS_ON_EVERY_EVENT); + } + + @Test + public void testLargeBufferedOutputInTimer() throws CannotProvideCoderException { + int maxResultsPerOutput = 100; + + // Array of sequences starting with 2 and the last element - 1. + // Output will be buffered until the last event arrives + long[] sequences = new long[maxResultsPerOutput * 3]; + for (int i = 0; i < sequences.length - 1; i++) { + sequences[i] = i + 2L; + } + sequences[sequences.length - 1] = 1; + + List events = new ArrayList<>(sequences.length); + Collection> expectedOutput = new ArrayList<>(sequences.length); + Collection> expectedStatuses = + new ArrayList<>(sequences.length + 10); + + StringBuilder output = new StringBuilder(); + String outputPerElement = "."; + String key = "id-1"; + + int bufferedEventCount = 0; + + for (long sequence : sequences) { + ++bufferedEventCount; + + events.add(Event.create(sequence, key, outputPerElement)); + output.append(outputPerElement); + expectedOutput.add(KV.of(key, output.toString())); + + if (bufferedEventCount < sequences.length) { + // Last event will result in a batch of events being produced. That's why it's excluded + // here. + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + null, bufferedEventCount, 2L, sequence, bufferedEventCount, 0L, 0, false))); + } + } + + // Statuses produced by the batched processing + for (int i = maxResultsPerOutput; i < sequences.length; i += maxResultsPerOutput) { + long lastOutputSequence = i; + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + lastOutputSequence, + sequences.length - lastOutputSequence, + lastOutputSequence + 1, + (long) sequences.length, + sequences.length, + lastOutputSequence, + 0, + false))); + } + + // -- Final status - indicates that everything has been fully processed + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + (long) sequences.length, + 0, + null, + null, + sequences.length, + sequences.length, + 0, + false))); + + testGlobalSequenceProcessing( + events.toArray(new Event[events.size()]), + expectedStatuses, + expectedOutput, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + 1L /* This dataset assumes 1 as the starting sequence */, + maxResultsPerOutput, + PRODUCE_STATUS_ON_EVERY_EVENT); + } + + @Test + public void testSequenceGapProcessingInBufferedOutput() throws CannotProvideCoderException { + int maxResultsPerOutput = 3; + + long[] sequences = new long[]{2, 3, 7, 8, 9, 10, 1, 4, 5, 6}; + + List events = new ArrayList<>(sequences.length); + List> expectedOutput = new ArrayList<>(sequences.length); + + StringBuilder output = new StringBuilder(); + String outputPerElement = "."; + String key = "id-1"; + + for (long sequence : sequences) { + events.add(Event.create(sequence, key, outputPerElement)); + output.append(outputPerElement); + expectedOutput.add(KV.of(key, output.toString())); + } + + int numberOfReceivedEvents = 0; + Collection> expectedStatuses = new ArrayList<>(); + + // First elements are out-of-sequence and they just get buffered. Earliest and latest sequence + // numbers keep changing. + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + null, 1, 2L, 2L, ++numberOfReceivedEvents, 0L, 0, false))); + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + null, 2, 2L, 3L, ++numberOfReceivedEvents, 0L, 0, false))); + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + null, 3, 2L, 7L, ++numberOfReceivedEvents, 0L, 0, false))); + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + null, 4, 2L, 8L, ++numberOfReceivedEvents, 0L, 0, false))); + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + null, 5, 2L, 9L, ++numberOfReceivedEvents, 0L, 0, false))); + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + null, 6, 2L, 10L, ++numberOfReceivedEvents, 0L, 0, false))); + // --- 1 has appeared and caused the batch to be sent out. + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + 3L, 4, 7L, 10L, ++numberOfReceivedEvents, 3L, 0, false))); + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + 4L, 4, 7L, 10L, ++numberOfReceivedEvents, 4L, 0, false))); + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + 5L, 4, 7L, 10L, ++numberOfReceivedEvents, 5L, 0, false))); + // --- 6 came and 6, 7, and 8 got output + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + 8L, 2, 9L, 10L, ++numberOfReceivedEvents, 8L, 0, false))); + // Last timer run produces the final status. Number of received events doesn't + // increase, + // this is the result of a timer processing + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + 10L, 0, null, null, numberOfReceivedEvents, 10L, 0, false))); + + testGlobalSequenceProcessing( + events.toArray(new Event[events.size()]), + expectedStatuses, + expectedOutput, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + 1L /* This dataset assumes 1 as the starting sequence */, + maxResultsPerOutput, + PRODUCE_STATUS_ON_EVERY_EVENT); + } + + @Test + public void testHandlingOfMaxSequenceNumber() throws CannotProvideCoderException { + Event[] events = { + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + Event.create(Long.MAX_VALUE, "id-1", "c") + }; + + Collection> expectedStatuses = new ArrayList<>(); + expectedStatuses.add( + KV.of("id-1", OrderedProcessingStatus.create(1L, 0, null, null, 3, 2, 0, false))); + + Collection> expectedOutput = new ArrayList<>(); + expectedOutput.add(KV.of("id-1", "a")); + expectedOutput.add(KV.of("id-1", "ab")); + + Collection>>> unprocessedEvents = + new ArrayList<>(); + unprocessedEvents.add( + KV.of( + "id-1", + KV.of( + Long.MAX_VALUE, + UnprocessedEvent.create("c", Reason.sequence_id_outside_valid_range)))); + + testGlobalSequenceProcessing( + events, + expectedStatuses, + expectedOutput, + unprocessedEvents, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT, + DONT_PRODUCE_STATUS_ON_EVERY_EVENT); + } + + @Test + public void testProcessingOfTheLastInput() throws CannotProvideCoderException { + Event[] events = { + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + Event.create(2, "id-1", StringEventExaminer.LAST_INPUT) + }; + + Collection> expectedStatuses = new ArrayList<>(); + expectedStatuses.add( + KV.of( + "id-1", + OrderedProcessingStatus.create( + 2L, 0, null, null, events.length, events.length, 0, LAST_EVENT_RECEIVED))); + + Collection> expectedOutput = new ArrayList<>(); + expectedOutput.add(KV.of("id-1", "a")); + expectedOutput.add(KV.of("id-1", "ab")); + expectedOutput.add(KV.of("id-1", "ab" + StringEventExaminer.LAST_INPUT)); + + testGlobalSequenceProcessing( + events, + expectedStatuses, + expectedOutput, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT, + DONT_PRODUCE_STATUS_ON_EVERY_EVENT); + } + + + private void testGlobalSequenceProcessing( + Event[] events, + Collection> expectedStatuses, + Collection> expectedOutput, + int emissionFrequency, + long initialSequence, + int maxResultsPerOutput, + boolean produceStatusOnEveryEvent) + throws CannotProvideCoderException { + testGlobalSequenceProcessing( + events, + expectedStatuses, + expectedOutput, + NO_EXPECTED_DLQ_EVENTS, + emissionFrequency, + initialSequence, + maxResultsPerOutput, + produceStatusOnEveryEvent); + } + + private void testGlobalSequenceProcessing( + Event[] events, + Collection> expectedStatuses, + Collection> expectedOutput, + Collection>>> expectedUnprocessedEvents, + int emissionFrequency, + long initialSequence, + int maxResultsPerOutput, + boolean produceStatusOnEveryEvent) + throws CannotProvideCoderException { + // Test a streaming pipeline + doTest( + events, + expectedStatuses, + expectedOutput, + expectedUnprocessedEvents, + emissionFrequency, + initialSequence, + maxResultsPerOutput, + produceStatusOnEveryEvent, + STREAMING, + SequenceType.GLOBAL); + + // Test a batch pipeline + doTest( + events, + expectedStatuses, + expectedOutput, + expectedUnprocessedEvents, + emissionFrequency, + initialSequence, + maxResultsPerOutput, + produceStatusOnEveryEvent, + BATCH, + SequenceType.GLOBAL); + } + + + @Test + public void testWindowedProcessing() throws CannotProvideCoderException { + + Instant base = new Instant(0); + TestStream values = + TestStream.create(streamingPipeline.getCoderRegistry().getCoder(Event.class)) + .advanceWatermarkTo(base) + .addElements( + // Start of first window + TimestampedValue.of( + Event.create(0, "id-1", "a"), base.plus(Duration.standardSeconds(1))), + TimestampedValue.of( + Event.create(1, "id-1", "b"), base.plus(Duration.standardSeconds(2))), + TimestampedValue.of( + Event.create(0, "id-2", "x"), base.plus(Duration.standardSeconds(1))), + TimestampedValue.of( + Event.create(1, "id-2", "y"), base.plus(Duration.standardSeconds(2))), + TimestampedValue.of( + Event.create(2, "id-2", "z"), base.plus(Duration.standardSeconds(2))), + + // Start of second window. Numbering must start with 0 again. + TimestampedValue.of( + Event.create(0, "id-1", "c"), base.plus(Duration.standardSeconds(10))), + TimestampedValue.of( + Event.create(1, "id-1", "d"), base.plus(Duration.standardSeconds(11)))) + .advanceWatermarkToInfinity(); + + Pipeline pipeline = streamingPipeline; + + PCollection rawInput = pipeline.apply("Create Streaming Events", values); + PCollection>> input = + rawInput.apply("To KV", ParDo.of(new MapEventsToKV())); + + input = input.apply("Window input", Window.into(FixedWindows.of(Duration.standardSeconds(5)))); + + StringBufferOrderedProcessingHandler handler = + new StringBufferOrderedProcessingHandler( + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0); + handler.setMaxOutputElementsPerBundle(LARGE_MAX_RESULTS_PER_OUTPUT); + handler.setStatusUpdateFrequency(null); + handler.setProduceStatusUpdateOnEveryEvent(true); + + OrderedEventProcessor orderedEventProcessor = + OrderedEventProcessor.create(handler); + + OrderedEventProcessorResult processingResult = + input.apply("Process Events", orderedEventProcessor); + + IntervalWindow window1 = new IntervalWindow(base, base.plus(Duration.standardSeconds(5))); + PAssert.that("Output matches in window 1", processingResult.output()) + .inWindow(window1) + .containsInAnyOrder( + KV.of("id-1", "a"), + KV.of("id-1", "ab"), + KV.of("id-2", "x"), + KV.of("id-2", "xy"), + KV.of("id-2", "xyz")); + + IntervalWindow window2 = + new IntervalWindow( + base.plus(Duration.standardSeconds(10)), base.plus(Duration.standardSeconds(15))); + PAssert.that("Output matches in window 2", processingResult.output()) + .inWindow(window2) + .containsInAnyOrder(KV.of("id-1", "c"), KV.of("id-1", "cd")); + + PAssert.that("Statuses match in window 1", processingResult.processingStatuses()) + .inWindow(window1) + .containsInAnyOrder( + KV.of("id-1", OrderedProcessingStatus.create(0L, 0, null, null, 1, 1, 0, false)), + KV.of("id-1", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2, 0, false)), + KV.of("id-2", OrderedProcessingStatus.create(0L, 0, null, null, 1, 1, 0, false)), + KV.of("id-2", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2, 0, false)), + KV.of("id-2", OrderedProcessingStatus.create(2L, 0, null, null, 3, 3, 0, false))); + + PAssert.that("Statuses match in window 2", processingResult.processingStatuses()) + .inWindow(window2) + .containsInAnyOrder( + KV.of("id-1", OrderedProcessingStatus.create(0L, 0, null, null, 1, 1, 0, false)), + KV.of("id-1", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2, 0, false))); + + PAssert.that("Unprocessed events match", processingResult.unprocessedEvents()) + .containsInAnyOrder(NO_EXPECTED_DLQ_EVENTS); + + pipeline.run(); + } +} diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java new file mode 100644 index 000000000000..37ce671ba574 --- /dev/null +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java @@ -0,0 +1,669 @@ +package org.apache.beam.sdk.extensions.ordered; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.extensions.ordered.UnprocessedEvent.Reason; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestStream; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Test; + +public class OrderedEventProcessorPerKeySequenceTest extends OrderedEventProcessorTest { + + @Test + public void testPerfectOrderingProcessing() throws CannotProvideCoderException { + Event[] events = { + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + Event.create(2, "id-1", "c"), + Event.create(3, "id-1", "d"), + Event.create(0, "id-2", "a"), + Event.create(1, "id-2", "b") + }; + + Collection> expectedStatuses = new ArrayList<>(); + expectedStatuses.add( + KV.of( + "id-1", + OrderedProcessingStatus.create( + 3L, + 0, + null, + null, + 4, + Arrays.stream(events).filter(e -> e.getKey().equals("id-1")).count(), + 0, + false))); + expectedStatuses.add( + KV.of( + "id-2", + OrderedProcessingStatus.create( + 1L, + 0, + null, + null, + 2, + Arrays.stream(events).filter(e -> e.getKey().equals("id-2")).count(), + 0, + false))); + + Collection> expectedOutput = new ArrayList<>(); + expectedOutput.add(KV.of("id-1", "a")); + expectedOutput.add(KV.of("id-1", "ab")); + expectedOutput.add(KV.of("id-1", "abc")); + expectedOutput.add(KV.of("id-1", "abcd")); + expectedOutput.add(KV.of("id-2", "a")); + expectedOutput.add(KV.of("id-2", "ab")); + + testPerKeySequenceProcessing( + events, + expectedStatuses, + expectedOutput, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT, + DONT_PRODUCE_STATUS_ON_EVERY_EVENT); + } + + @Test + public void testOutOfSequenceProcessing() throws CannotProvideCoderException { + Event[] events = { + Event.create(2, "id-1", "c"), + Event.create(1, "id-1", "b"), + Event.create(0, "id-1", "a"), + Event.create(3, "id-1", "d"), + Event.create(1, "id-2", "b"), + Event.create(2, "id-2", "c"), + Event.create(4, "id-2", "e"), + Event.create(0, "id-2", "a"), + Event.create(3, "id-2", "d") + }; + + Collection> expectedStatuses = new ArrayList<>(); + expectedStatuses.add( + KV.of( + "id-1", + OrderedProcessingStatus.create( + 3L, + 0, + null, + null, + 4, + Arrays.stream(events).filter(e -> e.getKey().equals("id-1")).count(), + 0, + false))); + expectedStatuses.add( + KV.of( + "id-2", + OrderedProcessingStatus.create( + 4L, + 0, + null, + null, + 5, + Arrays.stream(events).filter(e -> e.getKey().equals("id-2")).count(), + 0, + false))); + + Collection> expectedOutput = new ArrayList<>(); + expectedOutput.add(KV.of("id-1", "a")); + expectedOutput.add(KV.of("id-1", "ab")); + expectedOutput.add(KV.of("id-1", "abc")); + expectedOutput.add(KV.of("id-1", "abcd")); + expectedOutput.add(KV.of("id-2", "a")); + expectedOutput.add(KV.of("id-2", "ab")); + expectedOutput.add(KV.of("id-2", "abc")); + expectedOutput.add(KV.of("id-2", "abcd")); + expectedOutput.add(KV.of("id-2", "abcde")); + + testPerKeySequenceProcessing( + events, + expectedStatuses, + expectedOutput, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT, + DONT_PRODUCE_STATUS_ON_EVERY_EVENT); + } + + @Test + public void testUnfinishedProcessing() throws CannotProvideCoderException { + Event[] events = { + Event.create(2, "id-1", "c"), + // Excluded Event.create(1, "id-1", "b"), + Event.create(0, "id-1", "a"), + Event.create(3, "id-1", "d"), + Event.create(0, "id-2", "a"), + Event.create(1, "id-2", "b"), + }; + + Collection> expectedStatuses = new ArrayList<>(); + expectedStatuses.add( + KV.of("id-1", OrderedProcessingStatus.create(0L, 2, 2L, 3L, 3, 1L, 0, false))); + expectedStatuses.add( + KV.of("id-2", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2L, 0, false))); + + Collection> expectedOutput = new ArrayList<>(); + expectedOutput.add(KV.of("id-1", "a")); + expectedOutput.add(KV.of("id-2", "a")); + expectedOutput.add(KV.of("id-2", "ab")); + + testPerKeySequenceProcessing(events, expectedStatuses, expectedOutput, 1, 0, 1000, false); + } + + @Test + public void testHandlingOfDuplicateSequences() throws CannotProvideCoderException { + Event[] events = { + Event.create(3, "id-1", "d"), + Event.create(2, "id-1", "c"), + // Duplicates to be buffered + Event.create(3, "id-1", "d"), + Event.create(3, "id-1", "d"), + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + + // Duplicates after the events are processed + Event.create(1, "id-1", "b"), + Event.create(3, "id-1", "d"), + }; + int resultCount = 4; + int duplicateCount = 4; + + Collection> expectedStatuses = new ArrayList<>(); + expectedStatuses.add( + KV.of( + "id-1", + OrderedProcessingStatus.create( + 3L, 0, null, null, events.length, resultCount, duplicateCount, false))); + + Collection> expectedOutput = new ArrayList<>(); + expectedOutput.add(KV.of("id-1", "a")); + expectedOutput.add(KV.of("id-1", "ab")); + expectedOutput.add(KV.of("id-1", "abc")); + expectedOutput.add(KV.of("id-1", "abcd")); + + Collection>>> duplicates = new ArrayList<>(); + duplicates.add(KV.of("id-1", KV.of(3L, UnprocessedEvent.create("d", Reason.duplicate)))); + duplicates.add(KV.of("id-1", KV.of(3L, UnprocessedEvent.create("d", Reason.duplicate)))); + duplicates.add(KV.of("id-1", KV.of(1L, UnprocessedEvent.create("b", Reason.duplicate)))); + duplicates.add(KV.of("id-1", KV.of(3L, UnprocessedEvent.create("d", Reason.duplicate)))); + + testPerKeySequenceProcessing( + events, + expectedStatuses, + expectedOutput, + duplicates, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT, + DONT_PRODUCE_STATUS_ON_EVERY_EVENT); + } + + @Test + public void testHandlingOfCheckedExceptions() throws CannotProvideCoderException { + Event[] events = { + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + Event.create(2, "id-1", StringBuilderState.BAD_VALUE), + Event.create(3, "id-1", "c"), + }; + + Collection> expectedStatuses = new ArrayList<>(); + expectedStatuses.add( + KV.of("id-1", OrderedProcessingStatus.create(1L, 1, 3L, 3L, events.length, 2, 0, false))); + + Collection> expectedOutput = new ArrayList<>(); + expectedOutput.add(KV.of("id-1", "a")); + expectedOutput.add(KV.of("id-1", "ab")); + + Collection>>> failedEvents = new ArrayList<>(); + failedEvents.add( + KV.of( + "id-1", + KV.of( + 2L, + UnprocessedEvent.create(StringBuilderState.BAD_VALUE, Reason.exception_thrown)))); + + testPerKeySequenceProcessing( + events, + expectedStatuses, + expectedOutput, + failedEvents, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT, + DONT_PRODUCE_STATUS_ON_EVERY_EVENT); + } + + @Test + public void testProcessingWithEveryOtherResultEmission() throws CannotProvideCoderException { + Event[] events = { + Event.create(2, "id-1", "c"), + Event.create(1, "id-1", "b"), + Event.create(0, "id-1", "a"), + Event.create(3, "id-1", "d"), + Event.create(0, "id-2", "a"), + Event.create(1, "id-2", "b"), + }; + + Collection> expectedStatuses = new ArrayList<>(); + expectedStatuses.add( + KV.of("id-1", OrderedProcessingStatus.create(3L, 0, null, null, 4, 2L, 0, false))); + expectedStatuses.add( + KV.of("id-2", OrderedProcessingStatus.create(1L, 0, null, null, 2, 1L, 0, false))); + + Collection> expectedOutput = new ArrayList<>(); + expectedOutput.add(KV.of("id-1", "a")); + // Skipped KV.of("id-1", "ab"), + expectedOutput.add(KV.of("id-1", "abc")); + // Skipped KV.of("id-1", "abcd"), + expectedOutput.add(KV.of("id-2", "a")); + // Skipped KV.of("id-2", "ab") + testPerKeySequenceProcessing( + events, + expectedStatuses, + expectedOutput, + EMISSION_FREQUENCY_ON_EVERY_OTHER_EVENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT, + DONT_PRODUCE_STATUS_ON_EVERY_EVENT); + } + + @Test + public void testLargeBufferedOutputInTimer() throws CannotProvideCoderException { + int maxResultsPerOutput = 100; + + // Array of sequences starting with 2 and the last element - 1. + // Output will be buffered until the last event arrives + long[] sequences = new long[maxResultsPerOutput * 3]; + for (int i = 0; i < sequences.length - 1; i++) { + sequences[i] = i + 2L; + } + sequences[sequences.length - 1] = 1; + + List events = new ArrayList<>(sequences.length); + Collection> expectedOutput = new ArrayList<>(sequences.length); + Collection> expectedStatuses = + new ArrayList<>(sequences.length + 10); + + StringBuilder output = new StringBuilder(); + String outputPerElement = "."; + String key = "id-1"; + + int bufferedEventCount = 0; + + for (long sequence : sequences) { + ++bufferedEventCount; + + events.add(Event.create(sequence, key, outputPerElement)); + output.append(outputPerElement); + expectedOutput.add(KV.of(key, output.toString())); + + if (bufferedEventCount < sequences.length) { + // Last event will result in a batch of events being produced. That's why it's excluded + // here. + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + null, bufferedEventCount, 2L, sequence, bufferedEventCount, 0L, 0, false))); + } + } + + // Statuses produced by the batched processing + for (int i = maxResultsPerOutput; i < sequences.length; i += maxResultsPerOutput) { + long lastOutputSequence = i; + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + lastOutputSequence, + sequences.length - lastOutputSequence, + lastOutputSequence + 1, + (long) sequences.length, + sequences.length, + lastOutputSequence, + 0, + false))); + } + + // -- Final status - indicates that everything has been fully processed + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + (long) sequences.length, + 0, + null, + null, + sequences.length, + sequences.length, + 0, + false))); + + testPerKeySequenceProcessing( + events.toArray(new Event[events.size()]), + expectedStatuses, + expectedOutput, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + 1L /* This dataset assumes 1 as the starting sequence */, + maxResultsPerOutput, + PRODUCE_STATUS_ON_EVERY_EVENT); + } + + @Test + public void testSequenceGapProcessingInBufferedOutput() throws CannotProvideCoderException { + int maxResultsPerOutput = 3; + + long[] sequences = new long[] {2, 3, 7, 8, 9, 10, 1, 4, 5, 6}; + + List events = new ArrayList<>(sequences.length); + List> expectedOutput = new ArrayList<>(sequences.length); + + StringBuilder output = new StringBuilder(); + String outputPerElement = "."; + String key = "id-1"; + + for (long sequence : sequences) { + events.add(Event.create(sequence, key, outputPerElement)); + output.append(outputPerElement); + expectedOutput.add(KV.of(key, output.toString())); + } + + int numberOfReceivedEvents = 0; + Collection> expectedStatuses = new ArrayList<>(); + + // First elements are out-of-sequence and they just get buffered. Earliest and latest sequence + // numbers keep changing. + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + null, 1, 2L, 2L, ++numberOfReceivedEvents, 0L, 0, false))); + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + null, 2, 2L, 3L, ++numberOfReceivedEvents, 0L, 0, false))); + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + null, 3, 2L, 7L, ++numberOfReceivedEvents, 0L, 0, false))); + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + null, 4, 2L, 8L, ++numberOfReceivedEvents, 0L, 0, false))); + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + null, 5, 2L, 9L, ++numberOfReceivedEvents, 0L, 0, false))); + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + null, 6, 2L, 10L, ++numberOfReceivedEvents, 0L, 0, false))); + // --- 1 has appeared and caused the batch to be sent out. + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + 3L, 4, 7L, 10L, ++numberOfReceivedEvents, 3L, 0, false))); + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + 4L, 4, 7L, 10L, ++numberOfReceivedEvents, 4L, 0, false))); + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + 5L, 4, 7L, 10L, ++numberOfReceivedEvents, 5L, 0, false))); + // --- 6 came and 6, 7, and 8 got output + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + 8L, 2, 9L, 10L, ++numberOfReceivedEvents, 8L, 0, false))); + // Last timer run produces the final status. Number of received events doesn't + // increase, + // this is the result of a timer processing + expectedStatuses.add( + KV.of( + key, + OrderedProcessingStatus.create( + 10L, 0, null, null, numberOfReceivedEvents, 10L, 0, false))); + + testPerKeySequenceProcessing( + events.toArray(new Event[events.size()]), + expectedStatuses, + expectedOutput, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + 1L /* This dataset assumes 1 as the starting sequence */, + maxResultsPerOutput, + PRODUCE_STATUS_ON_EVERY_EVENT); + } + + @Test + public void testHandlingOfMaxSequenceNumber() throws CannotProvideCoderException { + Event[] events = { + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + Event.create(Long.MAX_VALUE, "id-1", "c") + }; + + Collection> expectedStatuses = new ArrayList<>(); + expectedStatuses.add( + KV.of("id-1", OrderedProcessingStatus.create(1L, 0, null, null, 3, 2, 0, false))); + + Collection> expectedOutput = new ArrayList<>(); + expectedOutput.add(KV.of("id-1", "a")); + expectedOutput.add(KV.of("id-1", "ab")); + + Collection>>> unprocessedEvents = + new ArrayList<>(); + unprocessedEvents.add( + KV.of( + "id-1", + KV.of( + Long.MAX_VALUE, + UnprocessedEvent.create("c", Reason.sequence_id_outside_valid_range)))); + + testPerKeySequenceProcessing( + events, + expectedStatuses, + expectedOutput, + unprocessedEvents, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT, + DONT_PRODUCE_STATUS_ON_EVERY_EVENT); + } + + @Test + public void testProcessingOfTheLastInput() throws CannotProvideCoderException { + Event[] events = { + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + Event.create(2, "id-1", StringEventExaminer.LAST_INPUT) + }; + + Collection> expectedStatuses = new ArrayList<>(); + expectedStatuses.add( + KV.of( + "id-1", + OrderedProcessingStatus.create( + 2L, 0, null, null, events.length, events.length, 0, LAST_EVENT_RECEIVED))); + + Collection> expectedOutput = new ArrayList<>(); + expectedOutput.add(KV.of("id-1", "a")); + expectedOutput.add(KV.of("id-1", "ab")); + expectedOutput.add(KV.of("id-1", "ab" + StringEventExaminer.LAST_INPUT)); + + testPerKeySequenceProcessing( + events, + expectedStatuses, + expectedOutput, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT, + DONT_PRODUCE_STATUS_ON_EVERY_EVENT); + } + + protected void testPerKeySequenceProcessing( + Event[] events, + Collection> expectedStatuses, + Collection> expectedOutput, + int emissionFrequency, + long initialSequence, + int maxResultsPerOutput, + boolean produceStatusOnEveryEvent) + throws CannotProvideCoderException { + testPerKeySequenceProcessing( + events, + expectedStatuses, + expectedOutput, + NO_EXPECTED_DLQ_EVENTS, + emissionFrequency, + initialSequence, + maxResultsPerOutput, + produceStatusOnEveryEvent); + } + + protected void testPerKeySequenceProcessing( + Event[] events, + Collection> expectedStatuses, + Collection> expectedOutput, + Collection>>> expectedUnprocessedEvents, + int emissionFrequency, + long initialSequence, + int maxResultsPerOutput, + boolean produceStatusOnEveryEvent) + throws CannotProvideCoderException { + // Test a streaming pipeline + doTest( + events, + expectedStatuses, + expectedOutput, + expectedUnprocessedEvents, + emissionFrequency, + initialSequence, + maxResultsPerOutput, + produceStatusOnEveryEvent, + STREAMING, + SequenceType.PER_KEY); + + // Test a batch pipeline + doTest( + events, + expectedStatuses, + expectedOutput, + expectedUnprocessedEvents, + emissionFrequency, + initialSequence, + maxResultsPerOutput, + produceStatusOnEveryEvent, + BATCH, + SequenceType.PER_KEY); + } + + @Test + public void testWindowedProcessing() throws CannotProvideCoderException { + + Instant base = new Instant(0); + TestStream values = + TestStream.create(streamingPipeline.getCoderRegistry().getCoder(Event.class)) + .advanceWatermarkTo(base) + .addElements( + // Start of first window + TimestampedValue.of( + Event.create(0, "id-1", "a"), base.plus(Duration.standardSeconds(1))), + TimestampedValue.of( + Event.create(1, "id-1", "b"), base.plus(Duration.standardSeconds(2))), + TimestampedValue.of( + Event.create(0, "id-2", "x"), base.plus(Duration.standardSeconds(1))), + TimestampedValue.of( + Event.create(1, "id-2", "y"), base.plus(Duration.standardSeconds(2))), + TimestampedValue.of( + Event.create(2, "id-2", "z"), base.plus(Duration.standardSeconds(2))), + + // Start of second window. Numbering must start with 0 again. + TimestampedValue.of( + Event.create(0, "id-1", "c"), base.plus(Duration.standardSeconds(10))), + TimestampedValue.of( + Event.create(1, "id-1", "d"), base.plus(Duration.standardSeconds(11)))) + .advanceWatermarkToInfinity(); + + Pipeline pipeline = streamingPipeline; + + PCollection rawInput = pipeline.apply("Create Streaming Events", values); + PCollection>> input = + rawInput.apply("To KV", ParDo.of(new MapEventsToKV())); + + input = input.apply("Window input", Window.into(FixedWindows.of(Duration.standardSeconds(5)))); + + StringBufferOrderedProcessingHandler handler = + new StringBufferOrderedProcessingHandler( + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0); + handler.setMaxOutputElementsPerBundle(LARGE_MAX_RESULTS_PER_OUTPUT); + handler.setStatusUpdateFrequency(null); + handler.setProduceStatusUpdateOnEveryEvent(true); + + OrderedEventProcessor orderedEventProcessor = + OrderedEventProcessor.create(handler); + + OrderedEventProcessorResult processingResult = + input.apply("Process Events", orderedEventProcessor); + + IntervalWindow window1 = new IntervalWindow(base, base.plus(Duration.standardSeconds(5))); + PAssert.that("Output matches in window 1", processingResult.output()) + .inWindow(window1) + .containsInAnyOrder( + KV.of("id-1", "a"), + KV.of("id-1", "ab"), + KV.of("id-2", "x"), + KV.of("id-2", "xy"), + KV.of("id-2", "xyz")); + + IntervalWindow window2 = + new IntervalWindow( + base.plus(Duration.standardSeconds(10)), base.plus(Duration.standardSeconds(15))); + PAssert.that("Output matches in window 2", processingResult.output()) + .inWindow(window2) + .containsInAnyOrder(KV.of("id-1", "c"), KV.of("id-1", "cd")); + + PAssert.that("Statuses match in window 1", processingResult.processingStatuses()) + .inWindow(window1) + .containsInAnyOrder( + KV.of("id-1", OrderedProcessingStatus.create(0L, 0, null, null, 1, 1, 0, false)), + KV.of("id-1", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2, 0, false)), + KV.of("id-2", OrderedProcessingStatus.create(0L, 0, null, null, 1, 1, 0, false)), + KV.of("id-2", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2, 0, false)), + KV.of("id-2", OrderedProcessingStatus.create(2L, 0, null, null, 3, 3, 0, false))); + + PAssert.that("Statuses match in window 2", processingResult.processingStatuses()) + .inWindow(window2) + .containsInAnyOrder( + KV.of("id-1", OrderedProcessingStatus.create(0L, 0, null, null, 1, 1, 0, false)), + KV.of("id-1", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2, 0, false))); + + PAssert.that("Unprocessed events match", processingResult.unprocessedEvents()) + .containsInAnyOrder(NO_EXPECTED_DLQ_EVENTS); + + pipeline.run(); + } +} diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTest.java index 6a24021ad667..1830b0f8a225 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTest.java @@ -17,11 +17,9 @@ */ package org.apache.beam.sdk.extensions.ordered; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.List; import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; @@ -36,14 +34,11 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Reshuffle; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; -import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; -import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.Repeatedly; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TimestampedValue; import org.checkerframework.checker.initialization.qual.Initialized; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.UnknownKeyFor; @@ -52,7 +47,6 @@ import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Rule; -import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -62,7 +56,6 @@ */ @RunWith(JUnit4.class) public class OrderedEventProcessorTest { - public static final boolean LAST_EVENT_RECEIVED = true; public static final int EMISSION_FREQUENCY_ON_EVERY_ELEMENT = 1; public static final int INITIAL_SEQUENCE_OF_0 = 0; @@ -74,8 +67,10 @@ public class OrderedEventProcessorTest { public static final boolean BATCH = false; public static final Set>>> NO_EXPECTED_DLQ_EVENTS = Collections.emptySet(); - @Rule public final transient TestPipeline streamingPipeline = TestPipeline.create(); - @Rule public final transient TestPipeline batchPipeline = TestPipeline.create(); + @Rule + public final transient TestPipeline streamingPipeline = TestPipeline.create(); + @Rule + public final transient TestPipeline batchPipeline = TestPipeline.create(); static class MapEventsToKV extends DoFn>> { @@ -97,646 +92,6 @@ public void map( } } - @Test - public void testPerfectOrderingProcessing() throws CannotProvideCoderException { - Event[] events = { - Event.create(0, "id-1", "a"), - Event.create(1, "id-1", "b"), - Event.create(2, "id-1", "c"), - Event.create(3, "id-1", "d"), - Event.create(0, "id-2", "a"), - Event.create(1, "id-2", "b") - }; - - Collection> expectedStatuses = new ArrayList<>(); - expectedStatuses.add( - KV.of( - "id-1", - OrderedProcessingStatus.create( - 3L, - 0, - null, - null, - 4, - Arrays.stream(events).filter(e -> e.getKey().equals("id-1")).count(), - 0, - false))); - expectedStatuses.add( - KV.of( - "id-2", - OrderedProcessingStatus.create( - 1L, - 0, - null, - null, - 2, - Arrays.stream(events).filter(e -> e.getKey().equals("id-2")).count(), - 0, - false))); - - Collection> expectedOutput = new ArrayList<>(); - expectedOutput.add(KV.of("id-1", "a")); - expectedOutput.add(KV.of("id-1", "ab")); - expectedOutput.add(KV.of("id-1", "abc")); - expectedOutput.add(KV.of("id-1", "abcd")); - expectedOutput.add(KV.of("id-2", "a")); - expectedOutput.add(KV.of("id-2", "ab")); - - testProcessing( - events, - expectedStatuses, - expectedOutput, - EMISSION_FREQUENCY_ON_EVERY_ELEMENT, - INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT, - DONT_PRODUCE_STATUS_ON_EVERY_EVENT); - } - - @Test - public void testOutOfSequenceProcessing() throws CannotProvideCoderException { - Event[] events = { - Event.create(2, "id-1", "c"), - Event.create(1, "id-1", "b"), - Event.create(0, "id-1", "a"), - Event.create(3, "id-1", "d"), - Event.create(1, "id-2", "b"), - Event.create(2, "id-2", "c"), - Event.create(4, "id-2", "e"), - Event.create(0, "id-2", "a"), - Event.create(3, "id-2", "d") - }; - - Collection> expectedStatuses = new ArrayList<>(); - expectedStatuses.add( - KV.of( - "id-1", - OrderedProcessingStatus.create( - 3L, - 0, - null, - null, - 4, - Arrays.stream(events).filter(e -> e.getKey().equals("id-1")).count(), - 0, - false))); - expectedStatuses.add( - KV.of( - "id-2", - OrderedProcessingStatus.create( - 4L, - 0, - null, - null, - 5, - Arrays.stream(events).filter(e -> e.getKey().equals("id-2")).count(), - 0, - false))); - - Collection> expectedOutput = new ArrayList<>(); - expectedOutput.add(KV.of("id-1", "a")); - expectedOutput.add(KV.of("id-1", "ab")); - expectedOutput.add(KV.of("id-1", "abc")); - expectedOutput.add(KV.of("id-1", "abcd")); - expectedOutput.add(KV.of("id-2", "a")); - expectedOutput.add(KV.of("id-2", "ab")); - expectedOutput.add(KV.of("id-2", "abc")); - expectedOutput.add(KV.of("id-2", "abcd")); - expectedOutput.add(KV.of("id-2", "abcde")); - - testProcessing( - events, - expectedStatuses, - expectedOutput, - EMISSION_FREQUENCY_ON_EVERY_ELEMENT, - INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT, - DONT_PRODUCE_STATUS_ON_EVERY_EVENT); - } - - @Test - public void testUnfinishedProcessing() throws CannotProvideCoderException { - Event[] events = { - Event.create(2, "id-1", "c"), - // Excluded Event.create(1, "id-1", "b"), - Event.create(0, "id-1", "a"), - Event.create(3, "id-1", "d"), - Event.create(0, "id-2", "a"), - Event.create(1, "id-2", "b"), - }; - - Collection> expectedStatuses = new ArrayList<>(); - expectedStatuses.add( - KV.of("id-1", OrderedProcessingStatus.create(0L, 2, 2L, 3L, 3, 1L, 0, false))); - expectedStatuses.add( - KV.of("id-2", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2L, 0, false))); - - Collection> expectedOutput = new ArrayList<>(); - expectedOutput.add(KV.of("id-1", "a")); - expectedOutput.add(KV.of("id-2", "a")); - expectedOutput.add(KV.of("id-2", "ab")); - - testProcessing(events, expectedStatuses, expectedOutput, 1, 0, 1000, false); - } - - @Test - public void testHandlingOfDuplicateSequences() throws CannotProvideCoderException { - Event[] events = { - Event.create(3, "id-1", "d"), - Event.create(2, "id-1", "c"), - // Duplicates to be buffered - Event.create(3, "id-1", "d"), - Event.create(3, "id-1", "d"), - Event.create(0, "id-1", "a"), - Event.create(1, "id-1", "b"), - - // Duplicates after the events are processed - Event.create(1, "id-1", "b"), - Event.create(3, "id-1", "d"), - }; - int resultCount = 4; - int duplicateCount = 4; - - Collection> expectedStatuses = new ArrayList<>(); - expectedStatuses.add( - KV.of( - "id-1", - OrderedProcessingStatus.create( - 3L, 0, null, null, events.length, resultCount, duplicateCount, false))); - - Collection> expectedOutput = new ArrayList<>(); - expectedOutput.add(KV.of("id-1", "a")); - expectedOutput.add(KV.of("id-1", "ab")); - expectedOutput.add(KV.of("id-1", "abc")); - expectedOutput.add(KV.of("id-1", "abcd")); - - Collection>>> duplicates = new ArrayList<>(); - duplicates.add(KV.of("id-1", KV.of(3L, UnprocessedEvent.create("d", Reason.duplicate)))); - duplicates.add(KV.of("id-1", KV.of(3L, UnprocessedEvent.create("d", Reason.duplicate)))); - duplicates.add(KV.of("id-1", KV.of(1L, UnprocessedEvent.create("b", Reason.duplicate)))); - duplicates.add(KV.of("id-1", KV.of(3L, UnprocessedEvent.create("d", Reason.duplicate)))); - - testProcessing( - events, - expectedStatuses, - expectedOutput, - duplicates, - EMISSION_FREQUENCY_ON_EVERY_ELEMENT, - INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT, - DONT_PRODUCE_STATUS_ON_EVERY_EVENT); - } - - @Test - public void testHandlingOfCheckedExceptions() throws CannotProvideCoderException { - Event[] events = { - Event.create(0, "id-1", "a"), - Event.create(1, "id-1", "b"), - Event.create(2, "id-1", StringBuilderState.BAD_VALUE), - Event.create(3, "id-1", "c"), - }; - - Collection> expectedStatuses = new ArrayList<>(); - expectedStatuses.add( - KV.of("id-1", OrderedProcessingStatus.create(1L, 1, 3L, 3L, events.length, 2, 0, false))); - - Collection> expectedOutput = new ArrayList<>(); - expectedOutput.add(KV.of("id-1", "a")); - expectedOutput.add(KV.of("id-1", "ab")); - - Collection>>> failedEvents = new ArrayList<>(); - failedEvents.add( - KV.of( - "id-1", - KV.of( - 2L, - UnprocessedEvent.create(StringBuilderState.BAD_VALUE, Reason.exception_thrown)))); - - testProcessing( - events, - expectedStatuses, - expectedOutput, - failedEvents, - EMISSION_FREQUENCY_ON_EVERY_ELEMENT, - INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT, - DONT_PRODUCE_STATUS_ON_EVERY_EVENT); - } - - @Test - public void testProcessingWithEveryOtherResultEmission() throws CannotProvideCoderException { - Event[] events = { - Event.create(2, "id-1", "c"), - Event.create(1, "id-1", "b"), - Event.create(0, "id-1", "a"), - Event.create(3, "id-1", "d"), - Event.create(0, "id-2", "a"), - Event.create(1, "id-2", "b"), - }; - - Collection> expectedStatuses = new ArrayList<>(); - expectedStatuses.add( - KV.of("id-1", OrderedProcessingStatus.create(3L, 0, null, null, 4, 2L, 0, false))); - expectedStatuses.add( - KV.of("id-2", OrderedProcessingStatus.create(1L, 0, null, null, 2, 1L, 0, false))); - - Collection> expectedOutput = new ArrayList<>(); - expectedOutput.add(KV.of("id-1", "a")); - // Skipped KV.of("id-1", "ab"), - expectedOutput.add(KV.of("id-1", "abc")); - // Skipped KV.of("id-1", "abcd"), - expectedOutput.add(KV.of("id-2", "a")); - // Skipped KV.of("id-2", "ab") - testProcessing( - events, - expectedStatuses, - expectedOutput, - EMISSION_FREQUENCY_ON_EVERY_OTHER_EVENT, - INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT, - DONT_PRODUCE_STATUS_ON_EVERY_EVENT); - } - - @Test - public void testLargeBufferedOutputInTimer() throws CannotProvideCoderException { - int maxResultsPerOutput = 100; - - // Array of sequences starting with 2 and the last element - 1. - // Output will be buffered until the last event arrives - long[] sequences = new long[maxResultsPerOutput * 3]; - for (int i = 0; i < sequences.length - 1; i++) { - sequences[i] = i + 2L; - } - sequences[sequences.length - 1] = 1; - - List events = new ArrayList<>(sequences.length); - Collection> expectedOutput = new ArrayList<>(sequences.length); - Collection> expectedStatuses = - new ArrayList<>(sequences.length + 10); - - StringBuilder output = new StringBuilder(); - String outputPerElement = "."; - String key = "id-1"; - - int bufferedEventCount = 0; - - for (long sequence : sequences) { - ++bufferedEventCount; - - events.add(Event.create(sequence, key, outputPerElement)); - output.append(outputPerElement); - expectedOutput.add(KV.of(key, output.toString())); - - if (bufferedEventCount < sequences.length) { - // Last event will result in a batch of events being produced. That's why it's excluded - // here. - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - null, bufferedEventCount, 2L, sequence, bufferedEventCount, 0L, 0, false))); - } - } - - // Statuses produced by the batched processing - for (int i = maxResultsPerOutput; i < sequences.length; i += maxResultsPerOutput) { - long lastOutputSequence = i; - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - lastOutputSequence, - sequences.length - lastOutputSequence, - lastOutputSequence + 1, - (long) sequences.length, - sequences.length, - lastOutputSequence, - 0, - false))); - } - - // -- Final status - indicates that everything has been fully processed - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - (long) sequences.length, - 0, - null, - null, - sequences.length, - sequences.length, - 0, - false))); - - testProcessing( - events.toArray(new Event[events.size()]), - expectedStatuses, - expectedOutput, - EMISSION_FREQUENCY_ON_EVERY_ELEMENT, - 1L /* This dataset assumes 1 as the starting sequence */, - maxResultsPerOutput, - PRODUCE_STATUS_ON_EVERY_EVENT); - } - - @Test - public void testSequenceGapProcessingInBufferedOutput() throws CannotProvideCoderException { - int maxResultsPerOutput = 3; - - long[] sequences = new long[] {2, 3, 7, 8, 9, 10, 1, 4, 5, 6}; - - List events = new ArrayList<>(sequences.length); - List> expectedOutput = new ArrayList<>(sequences.length); - - StringBuilder output = new StringBuilder(); - String outputPerElement = "."; - String key = "id-1"; - - for (long sequence : sequences) { - events.add(Event.create(sequence, key, outputPerElement)); - output.append(outputPerElement); - expectedOutput.add(KV.of(key, output.toString())); - } - - int numberOfReceivedEvents = 0; - Collection> expectedStatuses = new ArrayList<>(); - - // First elements are out-of-sequence and they just get buffered. Earliest and latest sequence - // numbers keep changing. - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - null, 1, 2L, 2L, ++numberOfReceivedEvents, 0L, 0, false))); - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - null, 2, 2L, 3L, ++numberOfReceivedEvents, 0L, 0, false))); - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - null, 3, 2L, 7L, ++numberOfReceivedEvents, 0L, 0, false))); - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - null, 4, 2L, 8L, ++numberOfReceivedEvents, 0L, 0, false))); - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - null, 5, 2L, 9L, ++numberOfReceivedEvents, 0L, 0, false))); - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - null, 6, 2L, 10L, ++numberOfReceivedEvents, 0L, 0, false))); - // --- 1 has appeared and caused the batch to be sent out. - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - 3L, 4, 7L, 10L, ++numberOfReceivedEvents, 3L, 0, false))); - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - 4L, 4, 7L, 10L, ++numberOfReceivedEvents, 4L, 0, false))); - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - 5L, 4, 7L, 10L, ++numberOfReceivedEvents, 5L, 0, false))); - // --- 6 came and 6, 7, and 8 got output - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - 8L, 2, 9L, 10L, ++numberOfReceivedEvents, 8L, 0, false))); - // Last timer run produces the final status. Number of received events doesn't - // increase, - // this is the result of a timer processing - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - 10L, 0, null, null, numberOfReceivedEvents, 10L, 0, false))); - - testProcessing( - events.toArray(new Event[events.size()]), - expectedStatuses, - expectedOutput, - EMISSION_FREQUENCY_ON_EVERY_ELEMENT, - 1L /* This dataset assumes 1 as the starting sequence */, - maxResultsPerOutput, - PRODUCE_STATUS_ON_EVERY_EVENT); - } - - @Test - public void testHandlingOfMaxSequenceNumber() throws CannotProvideCoderException { - Event[] events = { - Event.create(0, "id-1", "a"), - Event.create(1, "id-1", "b"), - Event.create(Long.MAX_VALUE, "id-1", "c") - }; - - Collection> expectedStatuses = new ArrayList<>(); - expectedStatuses.add( - KV.of("id-1", OrderedProcessingStatus.create(1L, 0, null, null, 3, 2, 0, false))); - - Collection> expectedOutput = new ArrayList<>(); - expectedOutput.add(KV.of("id-1", "a")); - expectedOutput.add(KV.of("id-1", "ab")); - - Collection>>> unprocessedEvents = - new ArrayList<>(); - unprocessedEvents.add( - KV.of( - "id-1", - KV.of( - Long.MAX_VALUE, - UnprocessedEvent.create("c", Reason.sequence_id_outside_valid_range)))); - - testProcessing( - events, - expectedStatuses, - expectedOutput, - unprocessedEvents, - EMISSION_FREQUENCY_ON_EVERY_ELEMENT, - INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT, - DONT_PRODUCE_STATUS_ON_EVERY_EVENT); - } - - @Test - public void testProcessingOfTheLastInput() throws CannotProvideCoderException { - Event[] events = { - Event.create(0, "id-1", "a"), - Event.create(1, "id-1", "b"), - Event.create(2, "id-1", StringEventExaminer.LAST_INPUT) - }; - - Collection> expectedStatuses = new ArrayList<>(); - expectedStatuses.add( - KV.of( - "id-1", - OrderedProcessingStatus.create( - 2L, 0, null, null, events.length, events.length, 0, LAST_EVENT_RECEIVED))); - - Collection> expectedOutput = new ArrayList<>(); - expectedOutput.add(KV.of("id-1", "a")); - expectedOutput.add(KV.of("id-1", "ab")); - expectedOutput.add(KV.of("id-1", "ab" + StringEventExaminer.LAST_INPUT)); - - testProcessing( - events, - expectedStatuses, - expectedOutput, - EMISSION_FREQUENCY_ON_EVERY_ELEMENT, - INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT, - DONT_PRODUCE_STATUS_ON_EVERY_EVENT); - } - - @Test - public void testWindowedProcessing() throws CannotProvideCoderException { - - Instant base = new Instant(0); - TestStream values = - TestStream.create(streamingPipeline.getCoderRegistry().getCoder(Event.class)) - .advanceWatermarkTo(base) - .addElements( - // Start of first window - TimestampedValue.of( - Event.create(0, "id-1", "a"), base.plus(Duration.standardSeconds(1))), - TimestampedValue.of( - Event.create(1, "id-1", "b"), base.plus(Duration.standardSeconds(2))), - TimestampedValue.of( - Event.create(0, "id-2", "x"), base.plus(Duration.standardSeconds(1))), - TimestampedValue.of( - Event.create(1, "id-2", "y"), base.plus(Duration.standardSeconds(2))), - TimestampedValue.of( - Event.create(2, "id-2", "z"), base.plus(Duration.standardSeconds(2))), - - // Start of second window. Numbering must start with 0 again. - TimestampedValue.of( - Event.create(0, "id-1", "c"), base.plus(Duration.standardSeconds(10))), - TimestampedValue.of( - Event.create(1, "id-1", "d"), base.plus(Duration.standardSeconds(11)))) - .advanceWatermarkToInfinity(); - - Pipeline pipeline = streamingPipeline; - - PCollection rawInput = pipeline.apply("Create Streaming Events", values); - PCollection>> input = - rawInput.apply("To KV", ParDo.of(new MapEventsToKV())); - - input = input.apply("Window input", Window.into(FixedWindows.of(Duration.standardSeconds(5)))); - - StringBufferOrderedProcessingHandler handler = - new StringBufferOrderedProcessingHandler( - EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0); - handler.setMaxOutputElementsPerBundle(LARGE_MAX_RESULTS_PER_OUTPUT); - handler.setStatusUpdateFrequency(null); - handler.setProduceStatusUpdateOnEveryEvent(true); - - OrderedEventProcessor orderedEventProcessor = - OrderedEventProcessor.create(handler); - - OrderedEventProcessorResult processingResult = - input.apply("Process Events", orderedEventProcessor); - - IntervalWindow window1 = new IntervalWindow(base, base.plus(Duration.standardSeconds(5))); - PAssert.that("Output matches in window 1", processingResult.output()) - .inWindow(window1) - .containsInAnyOrder( - KV.of("id-1", "a"), - KV.of("id-1", "ab"), - KV.of("id-2", "x"), - KV.of("id-2", "xy"), - KV.of("id-2", "xyz")); - - IntervalWindow window2 = - new IntervalWindow( - base.plus(Duration.standardSeconds(10)), base.plus(Duration.standardSeconds(15))); - PAssert.that("Output matches in window 2", processingResult.output()) - .inWindow(window2) - .containsInAnyOrder(KV.of("id-1", "c"), KV.of("id-1", "cd")); - - PAssert.that("Statuses match in window 1", processingResult.processingStatuses()) - .inWindow(window1) - .containsInAnyOrder( - KV.of("id-1", OrderedProcessingStatus.create(0L, 0, null, null, 1, 1, 0, false)), - KV.of("id-1", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2, 0, false)), - KV.of("id-2", OrderedProcessingStatus.create(0L, 0, null, null, 1, 1, 0, false)), - KV.of("id-2", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2, 0, false)), - KV.of("id-2", OrderedProcessingStatus.create(2L, 0, null, null, 3, 3, 0, false))); - - PAssert.that("Statuses match in window 2", processingResult.processingStatuses()) - .inWindow(window2) - .containsInAnyOrder( - KV.of("id-1", OrderedProcessingStatus.create(0L, 0, null, null, 1, 1, 0, false)), - KV.of("id-1", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2, 0, false))); - - PAssert.that("Unprocessed events match", processingResult.unprocessedEvents()) - .containsInAnyOrder(NO_EXPECTED_DLQ_EVENTS); - - pipeline.run(); - } - - private void testProcessing( - Event[] events, - Collection> expectedStatuses, - Collection> expectedOutput, - int emissionFrequency, - long initialSequence, - int maxResultsPerOutput, - boolean produceStatusOnEveryEvent) - throws CannotProvideCoderException { - testProcessing( - events, - expectedStatuses, - expectedOutput, - NO_EXPECTED_DLQ_EVENTS, - emissionFrequency, - initialSequence, - maxResultsPerOutput, - produceStatusOnEveryEvent); - } - - private void testProcessing( - Event[] events, - Collection> expectedStatuses, - Collection> expectedOutput, - Collection>>> expectedUnprocessedEvents, - int emissionFrequency, - long initialSequence, - int maxResultsPerOutput, - boolean produceStatusOnEveryEvent) - throws CannotProvideCoderException { - doTest( - events, - expectedStatuses, - expectedOutput, - expectedUnprocessedEvents, - emissionFrequency, - initialSequence, - maxResultsPerOutput, - produceStatusOnEveryEvent, - STREAMING); - doTest( - events, - expectedStatuses, - expectedOutput, - expectedUnprocessedEvents, - emissionFrequency, - initialSequence, - maxResultsPerOutput, - produceStatusOnEveryEvent, - BATCH); - } - /** * The majority of the tests use this method. Testing is done in the global window. * @@ -749,9 +104,10 @@ private void testProcessing( * @param maxResultsPerOutput * @param produceStatusOnEveryEvent * @param streaming + * @param sequenceType * @throws @UnknownKeyFor @NonNull @Initialized CannotProvideCoderException */ - private void doTest( + protected void doTest( Event[] events, Collection> expectedStatuses, Collection> expectedOutput, @@ -760,7 +116,8 @@ private void doTest( long initialSequence, int maxResultsPerOutput, boolean produceStatusOnEveryEvent, - boolean streaming) + boolean streaming, + SequenceType sequenceType) throws @UnknownKeyFor @NonNull @Initialized CannotProvideCoderException { Pipeline pipeline = streaming ? streamingPipeline : batchPipeline; @@ -783,6 +140,9 @@ private void doTest( handler.setStatusUpdateFrequency( streaming ? Duration.standardMinutes(5) : Duration.standardSeconds(1)); } + + handler.setSequenceType(sequenceType); + OrderedEventProcessor orderedEventProcessor = OrderedEventProcessor.create(handler); @@ -792,7 +152,7 @@ private void doTest( PAssert.that("Output matches", processingResult.output()).containsInAnyOrder(expectedOutput); if (streaming) { - // Only in streaming the events will arrive in a pre-determined order and the statuses + // Only in a streaming pipeline the events will arrive in a pre-determined order and the statuses // will be deterministic. In batch pipelines events can be processed in any order, // so we skip status verification and rely on the output and unprocessed event matches. PAssert.that("Statuses match", processingResult.processingStatuses()) @@ -884,7 +244,7 @@ public boolean matches(Object actual) { && originalEvent.getEvent().equals(eventToMatch.getEvent()) && originalEvent.getReason() == eventToMatch.getReason() && normalizeExplanation(originalEvent.getExplanation()) - .equals(normalizeExplanation(eventToMatch.getExplanation())); + .equals(normalizeExplanation(eventToMatch.getExplanation())); } @Override From 54bbd0766f097895f09b9dac70b40d08ab80d076 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Thu, 22 Aug 2024 14:19:40 -0700 Subject: [PATCH 02/33] Refactor the DoFns out of the transform and into a class hierarchy. --- .../ordered/GlobalSequencesProcessorDoFn.java | 40 ++ .../ordered/OrderedEventProcessor.java | 536 +----------------- .../ordered/OrderedProcessingStatus.java | 54 +- .../sdk/extensions/ordered/ProcessorDoFn.java | 157 +++++ .../ordered/SequencePerKeyProcessorDoFn.java | 431 ++++++++++++++ 5 files changed, 666 insertions(+), 552 deletions(-) create mode 100644 sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java create mode 100644 sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java create mode 100644 sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java new file mode 100644 index 000000000000..a1ba244445e9 --- /dev/null +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java @@ -0,0 +1,40 @@ +package org.apache.beam.sdk.extensions.ordered; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TupleTag; +import org.joda.time.Duration; + +class GlobalSequencesProcessorDoFn> + extends ProcessorDoFn { + + /** + * Stateful DoFn to do the bulk of processing. + * + * @param eventExaminer + * @param eventCoder + * @param stateCoder + * @param keyCoder + * @param mainOutputTupleTag + * @param statusTupleTag + * @param statusUpdateFrequency + * @param unprocessedEventTupleTag + * @param produceStatusUpdateOnEveryEvent + * @param maxNumberOfResultsToProduce + */ + GlobalSequencesProcessorDoFn(EventExaminer eventExaminer, + Coder eventCoder, + Coder stateCoder, + Coder keyCoder, + TupleTag> mainOutputTupleTag, + TupleTag> statusTupleTag, + Duration statusUpdateFrequency, + TupleTag>>> + unprocessedEventTupleTag, + boolean produceStatusUpdateOnEveryEvent, long maxNumberOfResultsToProduce) { + super(eventExaminer, mainOutputTupleTag, statusTupleTag, + statusUpdateFrequency, unprocessedEventTupleTag, produceStatusUpdateOnEveryEvent, + maxNumberOfResultsToProduce); + } +} diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java index adbc3987f9fe..198b2746209e 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java @@ -19,45 +19,28 @@ import com.google.auto.value.AutoValue; import java.util.Arrays; -import java.util.Iterator; import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.BooleanCoder; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.extensions.ordered.GlobalSequenceTracker.SequenceAndTimestamp; -import org.apache.beam.sdk.extensions.ordered.ProcessingState.ProcessingStateCoder; -import org.apache.beam.sdk.extensions.ordered.UnprocessedEvent.Reason; import org.apache.beam.sdk.extensions.ordered.UnprocessedEvent.UnprocessedEventCoder; import org.apache.beam.sdk.schemas.NoSuchSchemaException; import org.apache.beam.sdk.schemas.SchemaCoder; import org.apache.beam.sdk.schemas.SchemaRegistry; -import org.apache.beam.sdk.state.OrderedListState; -import org.apache.beam.sdk.state.StateSpec; -import org.apache.beam.sdk.state.StateSpecs; -import org.apache.beam.sdk.state.TimeDomain; -import org.apache.beam.sdk.state.Timer; -import org.apache.beam.sdk.state.TimerSpec; -import org.apache.beam.sdk.state.TimerSpecs; -import org.apache.beam.sdk.state.ValueState; 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.transforms.windowing.BoundedWindow; 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.PCollectionView; -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.TypeDescriptor; -import org.joda.time.Duration; import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Transform for processing ordered events. Events are grouped by the key and within each key they @@ -158,7 +141,7 @@ public void convert(@Element KV> element, processingResult = input.apply( ParDo.of( - new OrderedProcessorDoFn<>( + new GlobalSequencesProcessorDoFn( handler.getEventExaminer(), eventCoder, stateCoder, @@ -168,20 +151,19 @@ public void convert(@Element KV> element, handler.getStatusUpdateFrequency(), unprocessedEventOutput, handler.isProduceStatusUpdateOnEveryEvent(), - handler.getMaxOutputElementsPerBundle(), - latestContinousSequence)) + handler.getMaxOutputElementsPerBundle())) .withOutputTags( mainOutput, TupleTagList.of(Arrays.asList(statusOutput, unprocessedEventOutput))) .withSideInput(GLOBAL_SEQUENCE_TRACKER, latestContinousSequence) -); + ); break; case PER_KEY: processingResult = input.apply( ParDo.of( - new OrderedProcessorDoFn<>( + new SequencePerKeyProcessorDoFn( handler.getEventExaminer(), eventCoder, stateCoder, @@ -191,8 +173,7 @@ public void convert(@Element KV> element, handler.getStatusUpdateFrequency(), unprocessedEventOutput, handler.isProduceStatusUpdateOnEveryEvent(), - handler.getMaxOutputElementsPerBundle(), - null)) + handler.getMaxOutputElementsPerBundle())) .withOutputTags( mainOutput, TupleTagList.of(Arrays.asList(statusOutput, unprocessedEventOutput)))); @@ -234,511 +215,4 @@ private static Coder getOrderedProcessingStatusCoder(Pi return result; } - /** - * Main DoFn for processing ordered events. - * - * @param - * @param - * @param - */ - static class OrderedProcessorDoFn< - EventTypeT, - EventKeyTypeT, - ResultTypeT, - StateTypeT extends MutableState> - extends DoFn>, KV> { - - private static final Logger LOG = LoggerFactory.getLogger(OrderedProcessorDoFn.class); - - private static final String PROCESSING_STATE = "processingState"; - private static final String MUTABLE_STATE = "mutableState"; - private static final String BUFFERED_EVENTS = "bufferedEvents"; - private static final String STATUS_EMISSION_TIMER = "statusTimer"; - private static final String LARGE_BATCH_EMISSION_TIMER = "largeBatchTimer"; - private static final String WINDOW_CLOSED = "windowClosed"; - private final EventExaminer eventExaminer; - - @StateId(BUFFERED_EVENTS) - @SuppressWarnings("unused") - private final StateSpec> bufferedEventsSpec; - - @StateId(PROCESSING_STATE) - @SuppressWarnings("unused") - private final StateSpec>> processingStateSpec; - - @SuppressWarnings("unused") - @StateId(MUTABLE_STATE) - private final StateSpec> mutableStateSpec; - - @StateId(WINDOW_CLOSED) - @SuppressWarnings("unused") - private final StateSpec> windowClosedSpec; - - @TimerId(STATUS_EMISSION_TIMER) - @SuppressWarnings("unused") - private final TimerSpec statusEmissionTimer = TimerSpecs.timer(TimeDomain.PROCESSING_TIME); - - @TimerId(LARGE_BATCH_EMISSION_TIMER) - @SuppressWarnings("unused") - private final TimerSpec largeBatchEmissionTimer = TimerSpecs.timer(TimeDomain.EVENT_TIME); - - private final TupleTag> statusTupleTag; - private final Duration statusUpdateFrequency; - - private final TupleTag> mainOutputTupleTag; - private final TupleTag>>> - unprocessedEventsTupleTag; - private final boolean produceStatusUpdateOnEveryEvent; - - private final long maxNumberOfResultsToProduce; - - private final PCollectionView globalSequenceView; - - private Long numberOfResultsBeforeBundleStart; - - /** - * Stateful DoFn to do the bulk of processing. - * - * @param eventExaminer - * @param eventCoder - * @param stateCoder - * @param keyCoder - * @param mainOutputTupleTag - * @param statusTupleTag - * @param statusUpdateFrequency - * @param unprocessedEventTupleTag - * @param produceStatusUpdateOnEveryEvent - * @param maxNumberOfResultsToProduce - * @param globalSequenceView - */ - OrderedProcessorDoFn( - EventExaminer eventExaminer, - Coder eventCoder, - Coder stateCoder, - Coder keyCoder, - TupleTag> mainOutputTupleTag, - TupleTag> statusTupleTag, - Duration statusUpdateFrequency, - TupleTag>>> - unprocessedEventTupleTag, - boolean produceStatusUpdateOnEveryEvent, - long maxNumberOfResultsToProduce, - PCollectionView globalSequenceView) { - this.eventExaminer = eventExaminer; - this.bufferedEventsSpec = StateSpecs.orderedList(eventCoder); - this.mutableStateSpec = StateSpecs.value(stateCoder); - this.processingStateSpec = StateSpecs.value(ProcessingStateCoder.of(keyCoder)); - this.globalSequenceView = globalSequenceView; - this.windowClosedSpec = StateSpecs.value(BooleanCoder.of()); - this.mainOutputTupleTag = mainOutputTupleTag; - this.statusTupleTag = statusTupleTag; - this.unprocessedEventsTupleTag = unprocessedEventTupleTag; - this.statusUpdateFrequency = statusUpdateFrequency; - this.produceStatusUpdateOnEveryEvent = produceStatusUpdateOnEveryEvent; - this.maxNumberOfResultsToProduce = maxNumberOfResultsToProduce; - } - - @StartBundle - public void onBundleStart() { - numberOfResultsBeforeBundleStart = null; - } - - @FinishBundle - public void onBundleFinish() { - // This might be necessary because this field is also used in a Timer - numberOfResultsBeforeBundleStart = null; - } - - @ProcessElement - public void processElement( - @StateId(BUFFERED_EVENTS) OrderedListState bufferedEventsState, - @AlwaysFetched @StateId(PROCESSING_STATE) - ValueState> processingStateState, - @StateId(MUTABLE_STATE) ValueState mutableStateState, - @TimerId(STATUS_EMISSION_TIMER) Timer statusEmissionTimer, - @TimerId(LARGE_BATCH_EMISSION_TIMER) Timer largeBatchEmissionTimer, - @Element KV> eventAndSequence, - MultiOutputReceiver outputReceiver, - BoundedWindow window, - ProcessContext context) { - - if (globalSequenceView != null) { - SequenceAndTimestamp latestGlobalSequence = context.sideInput(globalSequenceView); - LOG.info("Latest sequence: " + latestGlobalSequence.getSequence()); - } - - EventKeyTypeT key = eventAndSequence.getKey(); - long sequence = eventAndSequence.getValue().getKey(); - EventTypeT event = eventAndSequence.getValue().getValue(); - - ProcessingState processingState = processingStateState.read(); - - if (processingState == null) { - // This is the first time we see this key/window pair - processingState = new ProcessingState<>(key); - if (statusUpdateFrequency != null) { - // Set up the timer to produce the status of the processing on a regular basis - statusEmissionTimer.offset(statusUpdateFrequency).setRelative(); - } - } - - if (numberOfResultsBeforeBundleStart == null) { - // Per key processing is synchronized by Beam. There is no need to have it here. - numberOfResultsBeforeBundleStart = processingState.getResultCount(); - } - - processingState.eventReceived(); - - StateTypeT state = - processNewEvent( - sequence, - event, - processingState, - mutableStateState, - bufferedEventsState, - outputReceiver); - - processBufferedEvents( - processingState, state, bufferedEventsState, outputReceiver, largeBatchEmissionTimer); - - saveStates( - processingStateState, - processingState, - mutableStateState, - state, - outputReceiver, - window.maxTimestamp()); - - checkIfProcessingIsCompleted(processingState); - } - - private boolean checkIfProcessingIsCompleted(ProcessingState processingState) { - boolean result = processingState.isProcessingCompleted(); - if (result) { - LOG.info("Processing for key '" + processingState.getKey() + "' is completed."); - } - return result; - } - - private void saveStates( - ValueState> processingStatusState, - ProcessingState processingStatus, - ValueState currentStateState, - StateTypeT state, - MultiOutputReceiver outputReceiver, - Instant windowTimestamp) { - // There is always a change to the processing status - processingStatusState.write(processingStatus); - - // Stored state may not have changes if the element was out of sequence. - if (state != null) { - currentStateState.write(state); - } - - if (produceStatusUpdateOnEveryEvent) { - // During pipeline draining the window timestamp is set to a large value in the future. - // Producing an event before that results in error, that's why this logic exist. - Instant statusTimestamp = windowTimestamp; - - emitProcessingStatus(processingStatus, outputReceiver, statusTimestamp); - } - } - - private void emitProcessingStatus( - ProcessingState processingState, - MultiOutputReceiver outputReceiver, - Instant statusTimestamp) { - outputReceiver - .get(statusTupleTag) - .outputWithTimestamp( - KV.of( - processingState.getKey(), - OrderedProcessingStatus.create( - processingState.getLastOutputSequence(), - processingState.getBufferedEventCount(), - processingState.getEarliestBufferedSequence(), - processingState.getLatestBufferedSequence(), - processingState.getEventsReceived(), - processingState.getResultCount(), - processingState.getDuplicates(), - processingState.isLastEventReceived())), - statusTimestamp); - } - - /** - * Process the just received event. - * - * @return newly created or updated State. If null is returned - the event wasn't processed. - */ - private StateTypeT processNewEvent( - long currentSequence, - EventTypeT currentEvent, - ProcessingState processingState, - ValueState currentStateState, - OrderedListState bufferedEventsState, - MultiOutputReceiver outputReceiver) { - if (currentSequence == Long.MAX_VALUE) { - // OrderedListState can't handle the timestamp based on MAX_VALUE. - // To avoid exceptions, we DLQ this event. - outputReceiver - .get(unprocessedEventsTupleTag) - .output( - KV.of( - processingState.getKey(), - KV.of( - currentSequence, - UnprocessedEvent.create( - currentEvent, Reason.sequence_id_outside_valid_range)))); - return null; - } - - if (processingState.hasAlreadyBeenProcessed(currentSequence)) { - outputReceiver - .get(unprocessedEventsTupleTag) - .output( - KV.of( - processingState.getKey(), - KV.of( - currentSequence, UnprocessedEvent.create(currentEvent, Reason.duplicate)))); - return null; - } - - StateTypeT state; - boolean thisIsTheLastEvent = eventExaminer.isLastEvent(currentSequence, currentEvent); - if (eventExaminer.isInitialEvent(currentSequence, currentEvent)) { - // First event of the key/window - // What if it's a duplicate event - it will reset everything. Shall we drop/DLQ anything - // that's before the processingState.lastOutputSequence? - state = eventExaminer.createStateOnInitialEvent(currentEvent); - - processingState.eventAccepted(currentSequence, thisIsTheLastEvent); - - ResultTypeT result = state.produceResult(); - if (result != null) { - outputReceiver.get(mainOutputTupleTag).output(KV.of(processingState.getKey(), result)); - processingState.resultProduced(); - } - - // Nothing else to do. We will attempt to process buffered events later. - return state; - } - - if (processingState.isNextEvent(currentSequence)) { - // Event matches expected sequence - state = currentStateState.read(); - - try { - state.mutate(currentEvent); - } catch (Exception e) { - outputReceiver - .get(unprocessedEventsTupleTag) - .output( - KV.of( - processingState.getKey(), - KV.of(currentSequence, UnprocessedEvent.create(currentEvent, e)))); - return null; - } - - ResultTypeT result = state.produceResult(); - if (result != null) { - outputReceiver.get(mainOutputTupleTag).output(KV.of(processingState.getKey(), result)); - processingState.resultProduced(); - } - processingState.eventAccepted(currentSequence, thisIsTheLastEvent); - - return state; - } - - // Event is not ready to be processed yet - Instant eventTimestamp = Instant.ofEpochMilli(currentSequence); - bufferedEventsState.add(TimestampedValue.of(currentEvent, eventTimestamp)); - processingState.eventBuffered(currentSequence, thisIsTheLastEvent); - - // This will signal that the state hasn't been mutated and we don't need to save it. - return null; - } - - /** - * Process buffered events. - */ - private void processBufferedEvents( - ProcessingState processingState, - StateTypeT state, - OrderedListState bufferedEventsState, - MultiOutputReceiver outputReceiver, - Timer largeBatchEmissionTimer) { - if (state == null) { - // Only when the current event caused a state mutation and the state is passed to this - // method should we attempt to process buffered events - return; - } - - if (!processingState.readyToProcessBufferedEvents()) { - return; - } - - if (reachedMaxResultCountForBundle(processingState, largeBatchEmissionTimer)) { - // No point in trying to process buffered events - return; - } - - Instant startRange = Instant.ofEpochMilli(processingState.getEarliestBufferedSequence()); - Instant endRange = Instant.ofEpochMilli(processingState.getLatestBufferedSequence() + 1); - Instant endClearRange = null; - - // readRange is efficiently implemented and will bring records in batches - Iterable> events = - bufferedEventsState.readRange(startRange, endRange); - - Iterator> bufferedEventsIterator = events.iterator(); - while (bufferedEventsIterator.hasNext()) { - TimestampedValue timestampedEvent = bufferedEventsIterator.next(); - Instant eventTimestamp = timestampedEvent.getTimestamp(); - long eventSequence = eventTimestamp.getMillis(); - - EventTypeT bufferedEvent = timestampedEvent.getValue(); - if (processingState.checkForDuplicateBatchedEvent(eventSequence)) { - outputReceiver - .get(unprocessedEventsTupleTag) - .output( - KV.of( - processingState.getKey(), - KV.of( - eventSequence, - UnprocessedEvent.create(bufferedEvent, Reason.duplicate)))); - continue; - } - - if (eventSequence > processingState.getLastOutputSequence() + 1) { - processingState.foundSequenceGap(eventSequence); - // Records will be cleared up to this element - endClearRange = Instant.ofEpochMilli(eventSequence); - break; - } - - // This check needs to be done after we checked for sequence gap and before we - // attempt to process the next element which can result in a new result. - if (reachedMaxResultCountForBundle(processingState, largeBatchEmissionTimer)) { - endClearRange = Instant.ofEpochMilli(eventSequence); - break; - } - - // Remove this record also - endClearRange = Instant.ofEpochMilli(eventSequence + 1); - - try { - state.mutate(bufferedEvent); - } catch (Exception e) { - outputReceiver - .get(unprocessedEventsTupleTag) - .output( - KV.of( - processingState.getKey(), - KV.of(eventSequence, UnprocessedEvent.create(bufferedEvent, e)))); - // There is a chance that the next event will have the same sequence number and will - // process successfully. - continue; - } - - ResultTypeT result = state.produceResult(); - if (result != null) { - outputReceiver.get(mainOutputTupleTag).output(KV.of(processingState.getKey(), result)); - processingState.resultProduced(); - } - processingState.processedBufferedEvent(eventSequence); - } - - bufferedEventsState.clearRange(startRange, endClearRange); - } - - private boolean reachedMaxResultCountForBundle( - ProcessingState processingState, Timer largeBatchEmissionTimer) { - boolean exceeded = - processingState.resultsProducedInBundle(numberOfResultsBeforeBundleStart) - >= maxNumberOfResultsToProduce; - if (exceeded) { - LOG.info( - "Setting the timer to output next batch of events for key '" - + processingState.getKey() - + "'"); - // See GroupIntoBatches for examples on how to hold the timestamp. - // TODO: test that on draining the pipeline all the results are still produced correctly. - // See: https://github.com/apache/beam/issues/30781 - largeBatchEmissionTimer.offset(Duration.millis(1)).setRelative(); - } - return exceeded; - } - - @OnTimer(LARGE_BATCH_EMISSION_TIMER) - public void onBatchEmission( - OnTimerContext context, - @StateId(BUFFERED_EVENTS) OrderedListState bufferedEventsState, - @AlwaysFetched @StateId(PROCESSING_STATE) - ValueState> processingStatusState, - @AlwaysFetched @StateId(MUTABLE_STATE) ValueState currentStateState, - @TimerId(LARGE_BATCH_EMISSION_TIMER) Timer largeBatchEmissionTimer, - MultiOutputReceiver outputReceiver) { - ProcessingState processingState = processingStatusState.read(); - if (processingState == null) { - LOG.warn("Processing state is empty. Ignore it if the pipeline is being cancelled."); - return; - } - StateTypeT state = currentStateState.read(); - if (state == null) { - LOG.warn("Mutable state is empty. Ignore it if the pipeline is being cancelled."); - return; - } - - LOG.debug("Starting to process batch for key '" + processingState.getKey() + "'"); - - this.numberOfResultsBeforeBundleStart = processingState.getResultCount(); - - processBufferedEvents( - processingState, state, bufferedEventsState, outputReceiver, largeBatchEmissionTimer); - - saveStates( - processingStatusState, - processingState, - currentStateState, - state, - outputReceiver, - // TODO: validate that this is correct. - context.window().maxTimestamp()); - - checkIfProcessingIsCompleted(processingState); - } - - @OnTimer(STATUS_EMISSION_TIMER) - @SuppressWarnings("unused") - public void onStatusEmission( - MultiOutputReceiver outputReceiver, - @TimerId(STATUS_EMISSION_TIMER) Timer statusEmissionTimer, - @StateId(WINDOW_CLOSED) ValueState windowClosedState, - @StateId(PROCESSING_STATE) - ValueState> processingStateState) { - - ProcessingState currentState = processingStateState.read(); - if (currentState == null) { - // This could happen if the state has been purged already during the draining. - // It means that there is nothing that we can do and we just need to return. - LOG.warn( - "Current processing state is null in onStatusEmission() - most likely the pipeline is shutting down."); - return; - } - - emitProcessingStatus(currentState, outputReceiver, Instant.now()); - - Boolean windowClosed = windowClosedState.read(); - if (!currentState.isProcessingCompleted() - // Stop producing statuses if we are finished for a particular key - && (windowClosed == null || !windowClosed)) { - statusEmissionTimer.offset(statusUpdateFrequency).setRelative(); - } - } - - @OnWindowExpiration - public void onWindowExpiration(@StateId(WINDOW_CLOSED) ValueState windowClosedState) { - windowClosedState.write(true); - } - } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingStatus.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingStatus.java index 6659bd2e2b92..de57ef46e1b9 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingStatus.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingStatus.java @@ -24,22 +24,24 @@ import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.joda.time.Instant; -/** Indicates the status of ordered processing for a particular key. */ +/** + * Indicates the status of ordered processing for a particular key. + */ @AutoValue @DefaultSchema(AutoValueSchema.class) public abstract class OrderedProcessingStatus { public static OrderedProcessingStatus create( - Long lastOutputSequence, + @Nullable Long lastProcessedSequence, long numberOfBufferedEvents, - Long earliestBufferedSequence, - Long latestBufferedSequence, + @Nullable Long earliestBufferedSequence, + @Nullable Long latestBufferedSequence, long numberOfReceivedEvents, long resultCount, long duplicateCount, boolean lastEventReceived) { return new AutoValue_OrderedProcessingStatus.Builder() - .setLastProcessedSequence(lastOutputSequence) + .setLastProcessedSequence(lastProcessedSequence) .setNumberOfBufferedEvents(numberOfBufferedEvents) .setEarliestBufferedSequence(earliestBufferedSequence) .setLatestBufferedSequence(latestBufferedSequence) @@ -53,39 +55,49 @@ public static OrderedProcessingStatus create( /** * @return Last sequence processed. If null is returned - no elements for the given key and window - * have been processed yet. + * have been processed yet. */ - @Nullable - public abstract Long getLastProcessedSequence(); - /** @return Number of events received out of sequence and buffered. */ + public abstract @Nullable Long getLastProcessedSequence(); + + /** + * @return Number of events received out of sequence and buffered. + */ public abstract long getNumberOfBufferedEvents(); - /** @return Earliest buffered sequence. If null is returned - there are no buffered events. */ + /** + * @return Earliest buffered sequence. If null is returned - there are no buffered events. + */ @Nullable public abstract Long getEarliestBufferedSequence(); - /** @return Latest buffered sequence. If null is returned - there are no buffered events. */ + /** + * @return Latest buffered sequence. If null is returned - there are no buffered events. + */ @Nullable public abstract Long getLatestBufferedSequence(); - /** @return Total number of events received for the given key and window. */ + /** + * @return Total number of events received for the given key and window. + */ public abstract long getNumberOfReceivedEvents(); /** - * @return Number of duplicate events which were output in {@link - * OrderedEventProcessorResult#unprocessedEvents()} PCollection + * @return Number of duplicate events which were output in + * {@link OrderedEventProcessorResult#unprocessedEvents()} PCollection */ public abstract long getDuplicateCount(); - /** @return Number of output results produced. */ + /** + * @return Number of output results produced. + */ public abstract long getResultCount(); /** * @return Indicator that the last event for the given key and window has been received. It - * doesn't necessarily mean that all the events for the given key and window have been - * processed. Use {@link OrderedProcessingStatus#getNumberOfBufferedEvents()} == 0 and this - * indicator as the sign that the processing is complete. + * doesn't necessarily mean that all the events for the given key and window have been processed. + * Use {@link OrderedProcessingStatus#getNumberOfBufferedEvents()} == 0 and this indicator as the + * sign that the processing is complete. */ public abstract boolean isLastEventReceived(); @@ -129,13 +141,13 @@ public final int hashCode() { @AutoValue.Builder public abstract static class Builder { - public abstract Builder setLastProcessedSequence(Long value); + public abstract Builder setLastProcessedSequence(@Nullable Long value); public abstract Builder setNumberOfBufferedEvents(long value); - public abstract Builder setEarliestBufferedSequence(Long value); + public abstract Builder setEarliestBufferedSequence(@Nullable Long value); - public abstract Builder setLatestBufferedSequence(Long value); + public abstract Builder setLatestBufferedSequence(@Nullable Long value); public abstract Builder setNumberOfReceivedEvents(long value); diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java new file mode 100644 index 000000000000..ee7843873d86 --- /dev/null +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java @@ -0,0 +1,157 @@ +package org.apache.beam.sdk.extensions.ordered; + +import org.apache.beam.sdk.state.Timer; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TupleTag; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Main DoFn for processing ordered events. + * + * @param + * @param + * @param + */ +abstract class ProcessorDoFn< + EventTypeT, + EventKeyTypeT, + ResultTypeT, + StateTypeT extends MutableState> + extends DoFn>, KV> { + + private static final Logger LOG = LoggerFactory.getLogger(ProcessorDoFn.class); + + protected static final String PROCESSING_STATE = "processingState"; + protected static final String MUTABLE_STATE = "mutableState"; + + protected static final String STATUS_EMISSION_TIMER = "statusTimer"; + protected static final String WINDOW_CLOSED = "windowClosed"; + protected final EventExaminer eventExaminer; + + private final TupleTag> statusTupleTag; + protected final Duration statusUpdateFrequency; + + protected final TupleTag> mainOutputTupleTag; + protected final TupleTag>>> + unprocessedEventsTupleTag; + private final boolean produceStatusUpdateOnEveryEvent; + + private final long maxNumberOfResultsToProduce; + + + protected @Nullable Long numberOfResultsBeforeBundleStart = Long.valueOf(0); + + /** + * Stateful DoFn to do the bulk of processing. + * + * @param eventExaminer + * @param mainOutputTupleTag + * @param statusTupleTag + * @param statusUpdateFrequency + * @param unprocessedEventTupleTag + * @param produceStatusUpdateOnEveryEvent + * @param maxNumberOfResultsToProduce + */ + ProcessorDoFn( + EventExaminer eventExaminer, + TupleTag> mainOutputTupleTag, + TupleTag> statusTupleTag, + Duration statusUpdateFrequency, + TupleTag>>> + unprocessedEventTupleTag, + boolean produceStatusUpdateOnEveryEvent, + long maxNumberOfResultsToProduce) { + this.eventExaminer = eventExaminer; + + this.mainOutputTupleTag = mainOutputTupleTag; + this.statusTupleTag = statusTupleTag; + this.unprocessedEventsTupleTag = unprocessedEventTupleTag; + this.statusUpdateFrequency = statusUpdateFrequency; + this.produceStatusUpdateOnEveryEvent = produceStatusUpdateOnEveryEvent; + this.maxNumberOfResultsToProduce = maxNumberOfResultsToProduce; + } + + @StartBundle + public void onBundleStart() { + numberOfResultsBeforeBundleStart = null; + } + + @FinishBundle + public void onBundleFinish() { + // This might be necessary because this field is also used in a Timer + numberOfResultsBeforeBundleStart = null; + } + + + protected void saveStates( + ValueState> processingStatusState, + ProcessingState processingStatus, + ValueState currentStateState, + @Nullable StateTypeT state, + MultiOutputReceiver outputReceiver, + Instant windowTimestamp) { + // There is always a change to the processing status + processingStatusState.write(processingStatus); + + // Stored state may not have changes if the element was out of sequence. + if (state != null) { + currentStateState.write(state); + } + + if (produceStatusUpdateOnEveryEvent) { + // During pipeline draining the window timestamp is set to a large value in the future. + // Producing an event before that results in error, that's why this logic exist. + Instant statusTimestamp = windowTimestamp; + + emitProcessingStatus(processingStatus, outputReceiver, statusTimestamp); + } + } + + protected void emitProcessingStatus( + ProcessingState processingState, + MultiOutputReceiver outputReceiver, + Instant statusTimestamp) { + outputReceiver + .get(statusTupleTag) + .outputWithTimestamp( + KV.of( + processingState.getKey(), + OrderedProcessingStatus.create( + processingState.getLastOutputSequence(), + processingState.getBufferedEventCount(), + processingState.getEarliestBufferedSequence(), + processingState.getLatestBufferedSequence(), + processingState.getEventsReceived(), + processingState.getResultCount(), + processingState.getDuplicates(), + processingState.isLastEventReceived())), + statusTimestamp); + } + + + protected boolean reachedMaxResultCountForBundle( + ProcessingState processingState, Timer largeBatchEmissionTimer) { + boolean exceeded = + processingState.resultsProducedInBundle( + numberOfResultsBeforeBundleStart == null ? 0 + : numberOfResultsBeforeBundleStart.longValue()) + >= maxNumberOfResultsToProduce; + if (exceeded) { + LOG.info( + "Setting the timer to output next batch of events for key '" + + processingState.getKey() + + "'"); + // See GroupIntoBatches for examples on how to hold the timestamp. + // TODO: test that on draining the pipeline all the results are still produced correctly. + // See: https://github.com/apache/beam/issues/30781 + largeBatchEmissionTimer.offset(Duration.millis(1)).setRelative(); + } + return exceeded; + } +} diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java new file mode 100644 index 000000000000..dd18c34f40c1 --- /dev/null +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java @@ -0,0 +1,431 @@ +package org.apache.beam.sdk.extensions.ordered; + +import java.util.Iterator; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.BooleanCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.extensions.ordered.ProcessingState.ProcessingStateCoder; +import org.apache.beam.sdk.extensions.ordered.UnprocessedEvent.Reason; +import org.apache.beam.sdk.state.OrderedListState; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.TimeDomain; +import org.apache.beam.sdk.state.Timer; +import org.apache.beam.sdk.state.TimerSpec; +import org.apache.beam.sdk.state.TimerSpecs; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.TupleTag; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class SequencePerKeyProcessorDoFn> + extends ProcessorDoFn { + private static final Logger LOG = LoggerFactory.getLogger(SequencePerKeyProcessorDoFn.class); + + private static final String LARGE_BATCH_EMISSION_TIMER = "largeBatchTimer"; + protected static final String BUFFERED_EVENTS = "bufferedEvents"; + @TimerId(LARGE_BATCH_EMISSION_TIMER) + @SuppressWarnings("unused") + private final TimerSpec largeBatchEmissionTimer = TimerSpecs.timer(TimeDomain.EVENT_TIME); + @StateId(BUFFERED_EVENTS) + @SuppressWarnings("unused") + private final StateSpec> bufferedEventsSpec; + @SuppressWarnings("unused") + @StateId(MUTABLE_STATE) + private final StateSpec> mutableStateSpec; + + @StateId(WINDOW_CLOSED) + @SuppressWarnings("unused") + private final StateSpec> windowClosedSpec; + + @TimerId(STATUS_EMISSION_TIMER) + @SuppressWarnings("unused") + private final TimerSpec statusEmissionTimer = TimerSpecs.timer(TimeDomain.PROCESSING_TIME); + @StateId(PROCESSING_STATE) + @SuppressWarnings("unused") + private final StateSpec>> processingStateSpec; + /** + * Stateful DoFn to do the bulk of processing. + * + * @param eventExaminer + * @param eventCoder + * @param stateCoder + * @param keyCoder + * @param mainOutputTupleTag + * @param statusTupleTag + * @param statusUpdateFrequency + * @param unprocessedEventTupleTag + * @param produceStatusUpdateOnEveryEvent + * @param maxNumberOfResultsToProduce + */ + SequencePerKeyProcessorDoFn( + EventExaminer eventExaminer, + Coder eventCoder, + Coder stateCoder, + Coder keyCoder, + TupleTag> mainOutputTupleTag, + TupleTag> statusTupleTag, + Duration statusUpdateFrequency, + TupleTag>>> + unprocessedEventTupleTag, + boolean produceStatusUpdateOnEveryEvent, long maxNumberOfResultsToProduce) { + super(eventExaminer, mainOutputTupleTag, statusTupleTag, + statusUpdateFrequency, unprocessedEventTupleTag, produceStatusUpdateOnEveryEvent, + maxNumberOfResultsToProduce); + this.bufferedEventsSpec = StateSpecs.orderedList(eventCoder); + this.processingStateSpec = StateSpecs.value(ProcessingStateCoder.of(keyCoder)); + this.mutableStateSpec = StateSpecs.value(stateCoder); + this.windowClosedSpec = StateSpecs.value(BooleanCoder.of()); + } + + + @ProcessElement + public void processElement( + @StateId(BUFFERED_EVENTS) OrderedListState bufferedEventsState, + @AlwaysFetched @StateId(PROCESSING_STATE) + ValueState> processingStateState, + @StateId(MUTABLE_STATE) ValueState mutableStateState, + @TimerId(STATUS_EMISSION_TIMER) Timer statusEmissionTimer, + @TimerId(LARGE_BATCH_EMISSION_TIMER) Timer largeBatchEmissionTimer, + @Element KV> eventAndSequence, + MultiOutputReceiver outputReceiver, + BoundedWindow window, + ProcessContext context) { + EventKeyTypeT key = eventAndSequence.getKey(); + long sequence = eventAndSequence.getValue().getKey(); + EventTypeT event = eventAndSequence.getValue().getValue(); + + ProcessingState processingState = processingStateState.read(); + + if (processingState == null) { + // This is the first time we see this key/window pair + processingState = new ProcessingState<>(key); + if (statusUpdateFrequency != null) { + // Set up the timer to produce the status of the processing on a regular basis + statusEmissionTimer.offset(statusUpdateFrequency).setRelative(); + } + } + + if (numberOfResultsBeforeBundleStart == null) { + // Per key processing is synchronized by Beam. There is no need to have it here. + numberOfResultsBeforeBundleStart = processingState.getResultCount(); + } + + processingState.eventReceived(); + + StateTypeT state = + processNewEvent( + sequence, + event, + processingState, + mutableStateState, + bufferedEventsState, + outputReceiver); + + processBufferedEvents( + processingState, state, bufferedEventsState, outputReceiver, largeBatchEmissionTimer); + + saveStates( + processingStateState, + processingState, + mutableStateState, + state, + outputReceiver, + window.maxTimestamp()); + + checkIfProcessingIsCompleted(processingState); + } + + private boolean checkIfProcessingIsCompleted(ProcessingState processingState) { + boolean result = processingState.isProcessingCompleted(); + if (result) { + LOG.info("Processing for key '" + processingState.getKey() + "' is completed."); + } + return result; + } + + /** + * Process the just received event. + * + * @return newly created or updated State. If null is returned - the event wasn't processed. + */ + private @Nullable StateTypeT processNewEvent( + long currentSequence, + EventTypeT currentEvent, + ProcessingState processingState, + ValueState currentStateState, + OrderedListState bufferedEventsState, + MultiOutputReceiver outputReceiver) { + if (currentSequence == Long.MAX_VALUE) { + // OrderedListState can't handle the timestamp based on MAX_VALUE. + // To avoid exceptions, we DLQ this event. + outputReceiver + .get(unprocessedEventsTupleTag) + .output( + KV.of( + processingState.getKey(), + KV.of( + currentSequence, + UnprocessedEvent.create( + currentEvent, Reason.sequence_id_outside_valid_range)))); + return null; + } + + if (processingState.hasAlreadyBeenProcessed(currentSequence)) { + outputReceiver + .get(unprocessedEventsTupleTag) + .output( + KV.of( + processingState.getKey(), + KV.of( + currentSequence, UnprocessedEvent.create(currentEvent, Reason.duplicate)))); + return null; + } + + StateTypeT state; + boolean thisIsTheLastEvent = eventExaminer.isLastEvent(currentSequence, currentEvent); + if (eventExaminer.isInitialEvent(currentSequence, currentEvent)) { + // First event of the key/window + // What if it's a duplicate event - it will reset everything. Shall we drop/DLQ anything + // that's before the processingState.lastOutputSequence? + state = eventExaminer.createStateOnInitialEvent(currentEvent); + + processingState.eventAccepted(currentSequence, thisIsTheLastEvent); + + ResultTypeT result = state.produceResult(); + if (result != null) { + outputReceiver.get(mainOutputTupleTag).output(KV.of(processingState.getKey(), result)); + processingState.resultProduced(); + } + + // Nothing else to do. We will attempt to process buffered events later. + return state; + } + + if (processingState.isNextEvent(currentSequence)) { + // Event matches expected sequence + state = currentStateState.read(); + if(state == null) { + LOG.warn("Unexpectedly got an empty state. Most likely cause is pipeline drainage."); + return null; + } + + try { + state.mutate(currentEvent); + } catch (Exception e) { + outputReceiver + .get(unprocessedEventsTupleTag) + .output( + KV.of( + processingState.getKey(), + KV.of(currentSequence, UnprocessedEvent.create(currentEvent, e)))); + return null; + } + + ResultTypeT result = state.produceResult(); + if (result != null) { + outputReceiver.get(mainOutputTupleTag).output(KV.of(processingState.getKey(), result)); + processingState.resultProduced(); + } + processingState.eventAccepted(currentSequence, thisIsTheLastEvent); + + return state; + } + + // Event is not ready to be processed yet + Instant eventTimestamp = Instant.ofEpochMilli(currentSequence); + bufferedEventsState.add(TimestampedValue.of(currentEvent, eventTimestamp)); + processingState.eventBuffered(currentSequence, thisIsTheLastEvent); + + // This will signal that the state hasn't been mutated and we don't need to save it. + return null; + } + + + /** + * Process buffered events. + */ + private void processBufferedEvents( + ProcessingState processingState, + @Nullable StateTypeT state, + OrderedListState bufferedEventsState, + MultiOutputReceiver outputReceiver, + Timer largeBatchEmissionTimer) { + if (state == null) { + // Only when the current event caused a state mutation and the state is passed to this + // method should we attempt to process buffered events + return; + } + + if (!processingState.readyToProcessBufferedEvents()) { + return; + } + + if (reachedMaxResultCountForBundle(processingState, largeBatchEmissionTimer)) { + // No point in trying to process buffered events + return; + } + + // Technically this block is not needed because these preconditions are checked + // earlier. Included to keep the linter happy. + Long earliestBufferedSequence = processingState.getEarliestBufferedSequence(); + if(earliestBufferedSequence == null) { + return; + } + Long latestBufferedSequence = processingState.getLatestBufferedSequence(); + if(latestBufferedSequence == null) { + return; + } + + Instant startRange = Instant.ofEpochMilli(earliestBufferedSequence); + Instant endRange = Instant.ofEpochMilli(latestBufferedSequence + 1); + Instant endClearRange = startRange; // it will get readjusted later. + + // readRange is efficiently implemented and will bring records in batches + Iterable> events = + bufferedEventsState.readRange(startRange, endRange); + + Iterator> bufferedEventsIterator = events.iterator(); + while (bufferedEventsIterator.hasNext()) { + TimestampedValue timestampedEvent = bufferedEventsIterator.next(); + Instant eventTimestamp = timestampedEvent.getTimestamp(); + long eventSequence = eventTimestamp.getMillis(); + + EventTypeT bufferedEvent = timestampedEvent.getValue(); + if (processingState.checkForDuplicateBatchedEvent(eventSequence)) { + outputReceiver + .get(unprocessedEventsTupleTag) + .output( + KV.of( + processingState.getKey(), + KV.of( + eventSequence, + UnprocessedEvent.create(bufferedEvent, Reason.duplicate)))); + continue; + } + + Long lastOutputSequence = processingState.getLastOutputSequence(); + if(lastOutputSequence == null) { + // Technically, should never happen. Included to keep the compiler happy + break; + } + if (eventSequence > lastOutputSequence + 1) { + processingState.foundSequenceGap(eventSequence); + // Records will be cleared up to this element + endClearRange = Instant.ofEpochMilli(eventSequence); + break; + } + + // This check needs to be done after we checked for sequence gap and before we + // attempt to process the next element which can result in a new result. + if (reachedMaxResultCountForBundle(processingState, largeBatchEmissionTimer)) { + endClearRange = Instant.ofEpochMilli(eventSequence); + break; + } + + // Remove this record also + endClearRange = Instant.ofEpochMilli(eventSequence + 1); + + try { + state.mutate(bufferedEvent); + } catch (Exception e) { + outputReceiver + .get(unprocessedEventsTupleTag) + .output( + KV.of( + processingState.getKey(), + KV.of(eventSequence, UnprocessedEvent.create(bufferedEvent, e)))); + // There is a chance that the next event will have the same sequence number and will + // process successfully. + continue; + } + + ResultTypeT result = state.produceResult(); + if (result != null) { + outputReceiver.get(mainOutputTupleTag).output(KV.of(processingState.getKey(), result)); + processingState.resultProduced(); + } + processingState.processedBufferedEvent(eventSequence); + } + + bufferedEventsState.clearRange(startRange, endClearRange); + } + + @OnTimer(LARGE_BATCH_EMISSION_TIMER) + public void onBatchEmission( + OnTimerContext context, + @StateId(BUFFERED_EVENTS) OrderedListState bufferedEventsState, + @AlwaysFetched @StateId(PROCESSING_STATE) + ValueState> processingStatusState, + @AlwaysFetched @StateId(MUTABLE_STATE) ValueState currentStateState, + @TimerId(LARGE_BATCH_EMISSION_TIMER) Timer largeBatchEmissionTimer, + MultiOutputReceiver outputReceiver) { + ProcessingState processingState = processingStatusState.read(); + if (processingState == null) { + LOG.warn("Processing state is empty. Ignore it if the pipeline is being cancelled."); + return; + } + StateTypeT state = currentStateState.read(); + if (state == null) { + LOG.warn("Mutable state is empty. Ignore it if the pipeline is being cancelled."); + return; + } + + LOG.debug("Starting to process batch for key '" + processingState.getKey() + "'"); + + this.numberOfResultsBeforeBundleStart = processingState.getResultCount(); + + processBufferedEvents( + processingState, state, bufferedEventsState, outputReceiver, largeBatchEmissionTimer); + + saveStates( + processingStatusState, + processingState, + currentStateState, + state, + outputReceiver, + // TODO: validate that this is correct. + context.window().maxTimestamp()); + + checkIfProcessingIsCompleted(processingState); + } + + @OnTimer(STATUS_EMISSION_TIMER) + @SuppressWarnings("unused") + public void onStatusEmission( + MultiOutputReceiver outputReceiver, + @TimerId(STATUS_EMISSION_TIMER) Timer statusEmissionTimer, + @StateId(WINDOW_CLOSED) ValueState windowClosedState, + @StateId(PROCESSING_STATE) + ValueState> processingStateState) { + + ProcessingState currentState = processingStateState.read(); + if (currentState == null) { + // This could happen if the state has been purged already during the draining. + // It means that there is nothing that we can do and we just need to return. + LOG.warn( + "Current processing state is null in onStatusEmission() - most likely the pipeline is shutting down."); + return; + } + + emitProcessingStatus(currentState, outputReceiver, Instant.now()); + + Boolean windowClosed = windowClosedState.read(); + if (!currentState.isProcessingCompleted() + // Stop producing statuses if we are finished for a particular key + && (windowClosed == null || !windowClosed)) { + statusEmissionTimer.offset(statusUpdateFrequency).setRelative(); + } + } + + @OnWindowExpiration + public void onWindowExpiration(@StateId(WINDOW_CLOSED) ValueState windowClosedState) { + windowClosedState.write(true); + } +} From 66e9e7ba0e0f18167164982eb11f344d2c992a87 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Fri, 23 Aug 2024 16:54:01 -0700 Subject: [PATCH 03/33] Next round of implementation of Global Sequence handling. --- .../ordered/GlobalSequencesProcessorDoFn.java | 214 ++++++++++++++++- .../ordered/OrderedEventProcessor.java | 4 +- .../extensions/ordered/ProcessingState.java | 24 +- .../sdk/extensions/ordered/ProcessorDoFn.java | 223 +++++++++++++++++- .../ordered/SequencePerKeyProcessorDoFn.java | 200 ++-------------- 5 files changed, 465 insertions(+), 200 deletions(-) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java index a1ba244445e9..4ed8c1891fb8 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java @@ -1,13 +1,61 @@ package org.apache.beam.sdk.extensions.ordered; +import org.apache.beam.sdk.coders.BooleanCoder; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.extensions.ordered.GlobalSequenceTracker.SequenceAndTimestamp; +import org.apache.beam.sdk.extensions.ordered.ProcessingState.ProcessingStateCoder; +import org.apache.beam.sdk.state.OrderedListState; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.TimeDomain; +import org.apache.beam.sdk.state.Timer; +import org.apache.beam.sdk.state.TimerSpec; +import org.apache.beam.sdk.state.TimerSpecs; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -class GlobalSequencesProcessorDoFn> - extends ProcessorDoFn { +class GlobalSequencesProcessorDoFn> + extends ProcessorDoFn { + + private static final Logger LOG = LoggerFactory.getLogger(GlobalSequencesProcessorDoFn.class); + + private static final String BATCH_EMISSION_TIMER = "batchTimer"; + + @TimerId(BATCH_EMISSION_TIMER) + @SuppressWarnings("unused") + private final TimerSpec batchTimerSpec = TimerSpecs.timer(TimeDomain.EVENT_TIME); + + private static final String BUFFERED_EVENTS = "bufferedEvents"; + @StateId(BUFFERED_EVENTS) + @SuppressWarnings("unused") + private final StateSpec> bufferedEventsSpec; + + @StateId(PROCESSING_STATE) + @SuppressWarnings("unused") + private final StateSpec>> processingStateSpec; + + @StateId(MUTABLE_STATE) + @SuppressWarnings("unused") + private final StateSpec> mutableStateSpec; + + @StateId(WINDOW_CLOSED) + @SuppressWarnings("unused") + private final StateSpec> windowClosedSpec; + + @TimerId(STATUS_EMISSION_TIMER) + @SuppressWarnings("unused") + private final TimerSpec statusEmissionTimer = TimerSpecs.timer(TimeDomain.PROCESSING_TIME); + + private final PCollectionView latestContinuousSequenceSideInput; /** * Stateful DoFn to do the bulk of processing. @@ -22,19 +70,163 @@ class GlobalSequencesProcessorDoFn eventExaminer, - Coder eventCoder, - Coder stateCoder, - Coder keyCoder, - TupleTag> mainOutputTupleTag, - TupleTag> statusTupleTag, + GlobalSequencesProcessorDoFn(EventExaminer eventExaminer, + Coder eventCoder, + Coder stateCoder, + Coder keyCoder, + TupleTag> mainOutputTupleTag, + TupleTag> statusTupleTag, Duration statusUpdateFrequency, - TupleTag>>> + TupleTag>>> unprocessedEventTupleTag, - boolean produceStatusUpdateOnEveryEvent, long maxNumberOfResultsToProduce) { + boolean produceStatusUpdateOnEveryEvent, long maxNumberOfResultsToProduce, + PCollectionView latestContinuousSequenceSideInput) { super(eventExaminer, mainOutputTupleTag, statusTupleTag, statusUpdateFrequency, unprocessedEventTupleTag, produceStatusUpdateOnEveryEvent, maxNumberOfResultsToProduce); + + this.latestContinuousSequenceSideInput = latestContinuousSequenceSideInput; + this.bufferedEventsSpec = StateSpecs.orderedList(eventCoder); + this.processingStateSpec = StateSpecs.value(ProcessingStateCoder.of(keyCoder)); + this.mutableStateSpec = StateSpecs.value(stateCoder); + this.windowClosedSpec = StateSpecs.value(BooleanCoder.of()); + } + + @Override + boolean checkForInitialEvent() { + return false; + } + + @Override + boolean checkForSequenceGapInBufferedEvents() { + return false; + } + + @ProcessElement + public void processElement(ProcessContext context, + @Element KV> eventAndSequence, + @StateId(BUFFERED_EVENTS) OrderedListState bufferedEventsState, + @AlwaysFetched @StateId(PROCESSING_STATE) + ValueState> processingStateState, + @StateId(MUTABLE_STATE) ValueState mutableStateState, + @TimerId(STATUS_EMISSION_TIMER) Timer statusEmissionTimer, + @TimerId(BATCH_EMISSION_TIMER) Timer batchEmissionTimer, + MultiOutputReceiver outputReceiver, + BoundedWindow window) { + + SequenceAndTimestamp lastContinuousSequence = context.sideInput( + latestContinuousSequenceSideInput); + LOG.info("lastSequence: " + lastContinuousSequence); + + EventKeyT key = eventAndSequence.getKey(); + long sequence = eventAndSequence.getValue().getKey(); + EventT event = eventAndSequence.getValue().getValue(); + + ProcessingState processingState = processingStateState.read(); + + if (processingState == null) { + // This is the first time we see this key/window pair + processingState = new ProcessingState<>(key); + if (statusUpdateFrequency != null) { + // Set up the timer to produce the status of the processing on a regular basis + statusEmissionTimer.offset(statusUpdateFrequency).setRelative(); + } + } + + processingState.updateGlobalSequenceDetails(lastContinuousSequence); + + if (numberOfResultsBeforeBundleStart == null) { + // Per key processing is synchronized by Beam. There is no need to have it here. + numberOfResultsBeforeBundleStart = processingState.getResultCount(); + } + + processingState.eventReceived(); + + StateT state = + processNewEvent( + sequence, + event, + processingState, + mutableStateState, + bufferedEventsState, + outputReceiver); + + saveStates( + processingStateState, + processingState, + mutableStateState, + state, + outputReceiver, + window.maxTimestamp()); + + // TODO: we can keep resetting this into the future under heavy load. + // Need to add logic to avoid doing it. + // + batchEmissionTimer.set(lastContinuousSequence.getTimestamp()); + + // Only if the record matches the sequence it can be output now + // TODO: refactor the code from SequencePerKeyDoFn + } + + @OnTimer(BATCH_EMISSION_TIMER) + public void onBatchEmission( + OnTimerContext context, + @StateId(BUFFERED_EVENTS) OrderedListState bufferedEventsState, + @AlwaysFetched @StateId(PROCESSING_STATE) + ValueState> processingStatusState, + @AlwaysFetched @StateId(MUTABLE_STATE) ValueState mutableStateState, + @TimerId(BATCH_EMISSION_TIMER) Timer batchEmissionTimer, + MultiOutputReceiver outputReceiver) { + + // At this point everything in the buffered state is ready to be processed up to the latest global sequence. + @Nullable ProcessingState processingState = processingStatusState.read(); + if(processingState == null) { + LOG.warn("Missing the processing state. Probably occurred during pipeline drainage"); + return; + } + + StateT state = mutableStateState.read(); + + Long lastCompleteGlobalSequence = processingState.getLastCompleteGlobalSequence(); + if(lastCompleteGlobalSequence == null) { + LOG.warn("Last complete global instance is null."); + return; + } + + Long earliestBufferedSequence = processingState.getEarliestBufferedSequence(); + if(earliestBufferedSequence == null) { + LOG.warn("Earliest buffered sequence is null."); + return; + } + Instant startRange = fromLong(earliestBufferedSequence); + Instant endRange = fromLong(lastCompleteGlobalSequence + 1); + + LOG.info("Processing range " + processingState.getKey() + " " + startRange + " " + endRange); + state = processBufferedEventRange(processingState, state, bufferedEventsState, outputReceiver, + batchEmissionTimer, startRange, endRange); + + saveStates( + processingStatusState, + processingState, + mutableStateState, + state, + outputReceiver, + // TODO: validate that this is correct. + context.window().maxTimestamp()); + } + + @OnTimer(STATUS_EMISSION_TIMER) + @SuppressWarnings("unused") + public void onStatusEmission( + MultiOutputReceiver outputReceiver, + @TimerId(STATUS_EMISSION_TIMER) Timer statusEmissionTimer, + @StateId(WINDOW_CLOSED) ValueState windowClosedState, + @StateId(PROCESSING_STATE) + ValueState> processingStateState) { + + processStatusTimerEvent(outputReceiver, statusEmissionTimer, windowClosedState, + processingStateState); } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java index 198b2746209e..6b014ddb796d 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java @@ -151,7 +151,9 @@ public void convert(@Element KV> element, handler.getStatusUpdateFrequency(), unprocessedEventOutput, handler.isProduceStatusUpdateOnEveryEvent(), - handler.getMaxOutputElementsPerBundle())) + handler.getMaxOutputElementsPerBundle(), + latestContinousSequence) + ) .withOutputTags( mainOutput, TupleTagList.of(Arrays.asList(statusOutput, unprocessedEventOutput))) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java index 4b591a37faab..8bc5488c0efc 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java @@ -28,6 +28,7 @@ import org.apache.beam.sdk.coders.NullableCoder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.extensions.ordered.GlobalSequenceTracker.SequenceAndTimestamp; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.initialization.qual.Initialized; @@ -51,6 +52,8 @@ class ProcessingState { private long resultCount; + @Nullable private Long lastCompleteGlobalSequence; + private KeyT key; public ProcessingState(KeyT key) { @@ -59,6 +62,7 @@ public ProcessingState(KeyT key) { this.lastOutputSequence = null; this.earliestBufferedSequence = null; this.latestBufferedSequence = null; + this.lastCompleteGlobalSequence = null; } /** @@ -130,6 +134,14 @@ public KeyT getKey() { return key; } + public @Nullable Long getLastCompleteGlobalSequence() { + return lastCompleteGlobalSequence; + } + + public void setLastCompleteGlobalSequence(@Nullable Long lastCompleteGlobalSequence) { + this.lastCompleteGlobalSequence = lastCompleteGlobalSequence; + } + /** * Current event matched the sequence and was processed. * @@ -274,6 +286,11 @@ public long resultsProducedInBundle(long numberOfResultsBeforeBundleStart) { return resultCount - numberOfResultsBeforeBundleStart; } + public void updateGlobalSequenceDetails(SequenceAndTimestamp updated) { + // TODO: do we need to select max? Do we care about the timestamp? + this.lastCompleteGlobalSequence = updated.getSequence(); + } + /** * Coder for the processing status. * @@ -308,6 +325,7 @@ public void encode(ProcessingState value, OutputStream outStream) throws I LONG_CODER.encode(value.getResultCount(), outStream); BOOLEAN_CODER.encode(value.isLastEventReceived(), outStream); keyCoder.encode(value.getKey(), outStream); + NULLABLE_LONG_CODER.encode(value.getLastCompleteGlobalSequence(), outStream); } @Override @@ -321,8 +339,9 @@ public ProcessingState decode(InputStream inStream) throws IOException { long resultCount = LONG_CODER.decode(inStream); boolean isLastEventReceived = BOOLEAN_CODER.decode(inStream); KeyT key = keyCoder.decode(inStream); + Long lastCompleteGlobalSequence = NULLABLE_LONG_CODER.decode(inStream); - return new ProcessingState<>( + ProcessingState result = new ProcessingState<>( key, lastOutputSequence, earliestBufferedSequence, @@ -332,6 +351,9 @@ public ProcessingState decode(InputStream inStream) throws IOException { duplicates, resultCount, isLastEventReceived); + result.setLastCompleteGlobalSequence(lastCompleteGlobalSequence); + + return result; } @Override diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java index ee7843873d86..b00b8e7b8c9c 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java @@ -1,11 +1,15 @@ package org.apache.beam.sdk.extensions.ordered; +import java.util.Iterator; +import javax.annotation.Nullable; +import org.apache.beam.sdk.extensions.ordered.UnprocessedEvent.Reason; +import org.apache.beam.sdk.state.OrderedListState; import org.apache.beam.sdk.state.Timer; import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; -import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; @@ -88,6 +92,105 @@ public void onBundleFinish() { numberOfResultsBeforeBundleStart = null; } + abstract boolean checkForInitialEvent(); + + /** + * Process the just received event. + * + * @return newly created or updated State. If null is returned - the event wasn't processed. + */ + protected @javax.annotation.Nullable StateTypeT processNewEvent( + long currentSequence, + EventTypeT currentEvent, + ProcessingState processingState, + ValueState currentStateState, + OrderedListState bufferedEventsState, + MultiOutputReceiver outputReceiver) { + if (currentSequence == Long.MAX_VALUE) { + // OrderedListState can't handle the timestamp based on MAX_VALUE. + // To avoid exceptions, we DLQ this event. + outputReceiver + .get(unprocessedEventsTupleTag) + .output( + KV.of( + processingState.getKey(), + KV.of( + currentSequence, + UnprocessedEvent.create( + currentEvent, Reason.sequence_id_outside_valid_range)))); + return null; + } + + if (processingState.hasAlreadyBeenProcessed(currentSequence)) { + outputReceiver + .get(unprocessedEventsTupleTag) + .output( + KV.of( + processingState.getKey(), + KV.of( + currentSequence, UnprocessedEvent.create(currentEvent, Reason.duplicate)))); + return null; + } + + StateTypeT state; + boolean thisIsTheLastEvent = eventExaminer.isLastEvent(currentSequence, currentEvent); + if (checkForInitialEvent() && eventExaminer.isInitialEvent(currentSequence, currentEvent)) { + // First event of the key/window + // What if it's a duplicate event - it will reset everything. Shall we drop/DLQ anything + // that's before the processingState.lastOutputSequence? + state = eventExaminer.createStateOnInitialEvent(currentEvent); + + processingState.eventAccepted(currentSequence, thisIsTheLastEvent); + + ResultTypeT result = state.produceResult(); + if (result != null) { + outputReceiver.get(mainOutputTupleTag).output(KV.of(processingState.getKey(), result)); + processingState.resultProduced(); + } + + // Nothing else to do. We will attempt to process buffered events later. + return state; + } + + if (processingState.isNextEvent(currentSequence)) { + // Event matches expected sequence + state = currentStateState.read(); + if(state == null) { + LOG.warn("Unexpectedly got an empty state. Most likely cause is pipeline drainage."); + return null; + } + + try { + state.mutate(currentEvent); + } catch (Exception e) { + outputReceiver + .get(unprocessedEventsTupleTag) + .output( + KV.of( + processingState.getKey(), + KV.of(currentSequence, UnprocessedEvent.create(currentEvent, e)))); + return null; + } + + ResultTypeT result = state.produceResult(); + if (result != null) { + outputReceiver.get(mainOutputTupleTag).output(KV.of(processingState.getKey(), result)); + processingState.resultProduced(); + } + processingState.eventAccepted(currentSequence, thisIsTheLastEvent); + + return state; + } + + // Event is not ready to be processed yet + bufferEvent(currentSequence, currentEvent, processingState, bufferedEventsState, + thisIsTheLastEvent); + + // This will signal that the state hasn't been mutated and we don't need to save it. + return null; + } + + protected void saveStates( ValueState> processingStatusState, @@ -113,6 +216,28 @@ protected void saveStates( } } + void processStatusTimerEvent(MultiOutputReceiver outputReceiver, Timer statusEmissionTimer, + ValueState windowClosedState, + ValueState> processingStateState) { + ProcessingState currentState = processingStateState.read(); + if (currentState == null) { + // This could happen if the state has been purged already during the draining. + // It means that there is nothing that we can do and we just need to return. + LOG.warn( + "Current processing state is null in onStatusEmission() - most likely the pipeline is shutting down."); + return; + } + + emitProcessingStatus(currentState, outputReceiver, Instant.now()); + + Boolean windowClosed = windowClosedState.read(); + if (!currentState.isProcessingCompleted() + // Stop producing statuses if we are finished for a particular key + && (windowClosed == null || !windowClosed)) { + statusEmissionTimer.offset(statusUpdateFrequency).setRelative(); + } + } + protected void emitProcessingStatus( ProcessingState processingState, MultiOutputReceiver outputReceiver, @@ -154,4 +279,100 @@ protected boolean reachedMaxResultCountForBundle( } return exceeded; } + + private void bufferEvent(long currentSequence, EventTypeT currentEvent, + ProcessingState processingState, + OrderedListState bufferedEventsState, boolean thisIsTheLastEvent) { + Instant eventTimestamp = fromLong(currentSequence); + bufferedEventsState.add(TimestampedValue.of(currentEvent, eventTimestamp)); + processingState.eventBuffered(currentSequence, thisIsTheLastEvent); + } + + abstract boolean checkForSequenceGapInBufferedEvents(); + + @Nullable StateTypeT processBufferedEventRange(ProcessingState processingState, + @Nullable StateTypeT state, + OrderedListState bufferedEventsState, MultiOutputReceiver outputReceiver, + Timer largeBatchEmissionTimer, Instant startRange, Instant endRange) { + // readRange is efficiently implemented and will bring records in batches + Iterable> events = + bufferedEventsState.readRange(startRange, endRange); + + Instant endClearRange = startRange; // it will get re-adjusted later. + + Iterator> bufferedEventsIterator = events.iterator(); + while (bufferedEventsIterator.hasNext()) { + TimestampedValue timestampedEvent = bufferedEventsIterator.next(); + Instant eventTimestamp = timestampedEvent.getTimestamp(); + long eventSequence = eventTimestamp.getMillis(); + + EventTypeT bufferedEvent = timestampedEvent.getValue(); + if (processingState.checkForDuplicateBatchedEvent(eventSequence)) { + outputReceiver + .get(unprocessedEventsTupleTag) + .output( + KV.of( + processingState.getKey(), + KV.of( + eventSequence, + UnprocessedEvent.create(bufferedEvent, Reason.duplicate)))); + continue; + } + + Long lastOutputSequence = processingState.getLastOutputSequence(); + if (checkForSequenceGapInBufferedEvents() && + lastOutputSequence != null && + eventSequence > lastOutputSequence + 1) { + processingState.foundSequenceGap(eventSequence); + // Records will be cleared up to this element + endClearRange = fromLong(eventSequence); + break; + } + + // This check needs to be done after we checked for sequence gap and before we + // attempt to process the next element which can result in a new result. + if (reachedMaxResultCountForBundle(processingState, largeBatchEmissionTimer)) { + endClearRange = fromLong(eventSequence); + break; + } + + // Remove this record also + endClearRange = fromLong(eventSequence + 1); + + try { + if(state == null) { + LOG.info("Creating a new state: " + processingState.getKey() + " " + bufferedEvent); + state = eventExaminer.createStateOnInitialEvent(bufferedEvent); + } else { + LOG.info("Mutating " + processingState.getKey() + " " + bufferedEvent); + state.mutate(bufferedEvent); + } + } catch (Exception e) { + outputReceiver + .get(unprocessedEventsTupleTag) + .output( + KV.of( + processingState.getKey(), + KV.of(eventSequence, UnprocessedEvent.create(bufferedEvent, e)))); + // There is a chance that the next event will have the same sequence number and will + // process successfully. + continue; + } + + ResultTypeT result = state.produceResult(); + if (result != null) { + outputReceiver.get(mainOutputTupleTag).output(KV.of(processingState.getKey(), result)); + processingState.resultProduced(); + } + processingState.processedBufferedEvent(eventSequence); + } + + bufferedEventsState.clearRange(startRange, endClearRange); + + return state; + } + + static Instant fromLong(long value) { + return Instant.ofEpochMilli(value); + } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java index dd18c34f40c1..26676370d954 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java @@ -84,6 +84,15 @@ class SequencePerKeyProcessorDoFn proc return result; } - /** - * Process the just received event. - * - * @return newly created or updated State. If null is returned - the event wasn't processed. - */ - private @Nullable StateTypeT processNewEvent( - long currentSequence, - EventTypeT currentEvent, - ProcessingState processingState, - ValueState currentStateState, - OrderedListState bufferedEventsState, - MultiOutputReceiver outputReceiver) { - if (currentSequence == Long.MAX_VALUE) { - // OrderedListState can't handle the timestamp based on MAX_VALUE. - // To avoid exceptions, we DLQ this event. - outputReceiver - .get(unprocessedEventsTupleTag) - .output( - KV.of( - processingState.getKey(), - KV.of( - currentSequence, - UnprocessedEvent.create( - currentEvent, Reason.sequence_id_outside_valid_range)))); - return null; - } - - if (processingState.hasAlreadyBeenProcessed(currentSequence)) { - outputReceiver - .get(unprocessedEventsTupleTag) - .output( - KV.of( - processingState.getKey(), - KV.of( - currentSequence, UnprocessedEvent.create(currentEvent, Reason.duplicate)))); - return null; - } - - StateTypeT state; - boolean thisIsTheLastEvent = eventExaminer.isLastEvent(currentSequence, currentEvent); - if (eventExaminer.isInitialEvent(currentSequence, currentEvent)) { - // First event of the key/window - // What if it's a duplicate event - it will reset everything. Shall we drop/DLQ anything - // that's before the processingState.lastOutputSequence? - state = eventExaminer.createStateOnInitialEvent(currentEvent); - - processingState.eventAccepted(currentSequence, thisIsTheLastEvent); - - ResultTypeT result = state.produceResult(); - if (result != null) { - outputReceiver.get(mainOutputTupleTag).output(KV.of(processingState.getKey(), result)); - processingState.resultProduced(); - } - - // Nothing else to do. We will attempt to process buffered events later. - return state; - } - - if (processingState.isNextEvent(currentSequence)) { - // Event matches expected sequence - state = currentStateState.read(); - if(state == null) { - LOG.warn("Unexpectedly got an empty state. Most likely cause is pipeline drainage."); - return null; - } - - try { - state.mutate(currentEvent); - } catch (Exception e) { - outputReceiver - .get(unprocessedEventsTupleTag) - .output( - KV.of( - processingState.getKey(), - KV.of(currentSequence, UnprocessedEvent.create(currentEvent, e)))); - return null; - } - - ResultTypeT result = state.produceResult(); - if (result != null) { - outputReceiver.get(mainOutputTupleTag).output(KV.of(processingState.getKey(), result)); - processingState.resultProduced(); - } - processingState.eventAccepted(currentSequence, thisIsTheLastEvent); - - return state; - } - - // Event is not ready to be processed yet - Instant eventTimestamp = Instant.ofEpochMilli(currentSequence); - bufferedEventsState.add(TimestampedValue.of(currentEvent, eventTimestamp)); - processingState.eventBuffered(currentSequence, thisIsTheLastEvent); - - // This will signal that the state hasn't been mutated and we don't need to save it. - return null; - } - - /** * Process buffered events. */ @@ -283,78 +194,11 @@ private void processBufferedEvents( return; } - Instant startRange = Instant.ofEpochMilli(earliestBufferedSequence); - Instant endRange = Instant.ofEpochMilli(latestBufferedSequence + 1); - Instant endClearRange = startRange; // it will get readjusted later. - - // readRange is efficiently implemented and will bring records in batches - Iterable> events = - bufferedEventsState.readRange(startRange, endRange); - - Iterator> bufferedEventsIterator = events.iterator(); - while (bufferedEventsIterator.hasNext()) { - TimestampedValue timestampedEvent = bufferedEventsIterator.next(); - Instant eventTimestamp = timestampedEvent.getTimestamp(); - long eventSequence = eventTimestamp.getMillis(); - - EventTypeT bufferedEvent = timestampedEvent.getValue(); - if (processingState.checkForDuplicateBatchedEvent(eventSequence)) { - outputReceiver - .get(unprocessedEventsTupleTag) - .output( - KV.of( - processingState.getKey(), - KV.of( - eventSequence, - UnprocessedEvent.create(bufferedEvent, Reason.duplicate)))); - continue; - } - - Long lastOutputSequence = processingState.getLastOutputSequence(); - if(lastOutputSequence == null) { - // Technically, should never happen. Included to keep the compiler happy - break; - } - if (eventSequence > lastOutputSequence + 1) { - processingState.foundSequenceGap(eventSequence); - // Records will be cleared up to this element - endClearRange = Instant.ofEpochMilli(eventSequence); - break; - } + Instant startRange = fromLong(earliestBufferedSequence); + Instant endRange = fromLong(latestBufferedSequence + 1); - // This check needs to be done after we checked for sequence gap and before we - // attempt to process the next element which can result in a new result. - if (reachedMaxResultCountForBundle(processingState, largeBatchEmissionTimer)) { - endClearRange = Instant.ofEpochMilli(eventSequence); - break; - } - - // Remove this record also - endClearRange = Instant.ofEpochMilli(eventSequence + 1); - - try { - state.mutate(bufferedEvent); - } catch (Exception e) { - outputReceiver - .get(unprocessedEventsTupleTag) - .output( - KV.of( - processingState.getKey(), - KV.of(eventSequence, UnprocessedEvent.create(bufferedEvent, e)))); - // There is a chance that the next event will have the same sequence number and will - // process successfully. - continue; - } - - ResultTypeT result = state.produceResult(); - if (result != null) { - outputReceiver.get(mainOutputTupleTag).output(KV.of(processingState.getKey(), result)); - processingState.resultProduced(); - } - processingState.processedBufferedEvent(eventSequence); - } - - bufferedEventsState.clearRange(startRange, endClearRange); + processBufferedEventRange(processingState, state, bufferedEventsState, outputReceiver, largeBatchEmissionTimer, + startRange, endRange); } @OnTimer(LARGE_BATCH_EMISSION_TIMER) @@ -405,23 +249,7 @@ public void onStatusEmission( @StateId(PROCESSING_STATE) ValueState> processingStateState) { - ProcessingState currentState = processingStateState.read(); - if (currentState == null) { - // This could happen if the state has been purged already during the draining. - // It means that there is nothing that we can do and we just need to return. - LOG.warn( - "Current processing state is null in onStatusEmission() - most likely the pipeline is shutting down."); - return; - } - - emitProcessingStatus(currentState, outputReceiver, Instant.now()); - - Boolean windowClosed = windowClosedState.read(); - if (!currentState.isProcessingCompleted() - // Stop producing statuses if we are finished for a particular key - && (windowClosed == null || !windowClosed)) { - statusEmissionTimer.offset(statusUpdateFrequency).setRelative(); - } + processStatusTimerEvent(outputReceiver, statusEmissionTimer, windowClosedState, processingStateState); } @OnWindowExpiration From 66490da9c8fc4c834fa4de70a326f36e3ac87f12 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Mon, 26 Aug 2024 15:41:25 -0700 Subject: [PATCH 04/33] Added ticker timers in global sequence processing. --- .../ordered/GlobalSequencesProcessorDoFn.java | 15 ++- .../ordered/OrderedEventProcessor.java | 79 +++++++------ .../ordered/PerKeyTickerGenerator.java | 110 ++++++++++++++++++ 3 files changed, 167 insertions(+), 37 deletions(-) create mode 100644 sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/PerKeyTickerGenerator.java diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java index 4ed8c1891fb8..08d72c2eb454 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java @@ -120,9 +120,10 @@ public void processElement(ProcessContext context, latestContinuousSequenceSideInput); LOG.info("lastSequence: " + lastContinuousSequence); + EventT event = eventAndSequence.getValue().getValue(); + EventKeyT key = eventAndSequence.getKey(); long sequence = eventAndSequence.getValue().getKey(); - EventT event = eventAndSequence.getValue().getValue(); ProcessingState processingState = processingStateState.read(); @@ -137,6 +138,12 @@ public void processElement(ProcessContext context, processingState.updateGlobalSequenceDetails(lastContinuousSequence); + if (event == null) { + // This is the ticker event. We only need to update the state as it relates to the global sequence. + processingStateState.write(processingState); + return; + } + if (numberOfResultsBeforeBundleStart == null) { // Per key processing is synchronized by Beam. There is no need to have it here. numberOfResultsBeforeBundleStart = processingState.getResultCount(); @@ -182,7 +189,7 @@ public void onBatchEmission( // At this point everything in the buffered state is ready to be processed up to the latest global sequence. @Nullable ProcessingState processingState = processingStatusState.read(); - if(processingState == null) { + if (processingState == null) { LOG.warn("Missing the processing state. Probably occurred during pipeline drainage"); return; } @@ -190,13 +197,13 @@ public void onBatchEmission( StateT state = mutableStateState.read(); Long lastCompleteGlobalSequence = processingState.getLastCompleteGlobalSequence(); - if(lastCompleteGlobalSequence == null) { + if (lastCompleteGlobalSequence == null) { LOG.warn("Last complete global instance is null."); return; } Long earliestBufferedSequence = processingState.getEarliestBufferedSequence(); - if(earliestBufferedSequence == null) { + if (earliestBufferedSequence == null) { LOG.warn("Earliest buffered sequence is null."); return; } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java index 6b014ddb796d..2b49ef9cee24 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java @@ -31,15 +31,18 @@ import org.apache.beam.sdk.schemas.SchemaCoder; import org.apache.beam.sdk.schemas.SchemaRegistry; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; 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.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.TypeDescriptor; +import org.joda.time.Duration; import org.joda.time.Instant; /** @@ -123,42 +126,41 @@ public OrderedEventProcessorResult expand( switch (handler.getSequenceType()) { case GLOBAL: - DoFn>, SequenceAndTimestamp> fn = new DoFn>, SequenceAndTimestamp>() { - @ProcessElement - public void convert(@Element KV> element, - @Timestamp Instant timestamp, - OutputReceiver outputReceiver - ) { - outputReceiver.output( - SequenceAndTimestamp.create(element.getValue().getKey(), timestamp)); - } - }; - - final PCollectionView latestContinousSequence = + final PCollectionView latestContinuousSequence = input - .apply("Convert to SequenceAndTimestamp", ParDo.of(fn)) - .apply("GlobalSequenceTracker", new GlobalSequenceTracker()); + .apply("Convert to SequenceAndTimestamp", + ParDo.of(new EventToSequenceAndTimestampConverter<>())) + .apply("Global Sequence Tracker", new GlobalSequenceTracker()); + + PCollection>> tickers = input.apply("Create Tickers", + new PerKeyTickerGenerator<>(keyCoder, eventCoder, Duration.standardSeconds(5))); + + PCollection>> eventsAndTickers = + PCollectionList.of(input).and(tickers) + .apply("Combine Events and Tickers", Flatten.pCollections()) + .setCoder(tickers.getCoder()); processingResult = - input.apply( - ParDo.of( - new GlobalSequencesProcessorDoFn( - handler.getEventExaminer(), - eventCoder, - stateCoder, - keyCoder, - mainOutput, - statusOutput, - handler.getStatusUpdateFrequency(), - unprocessedEventOutput, - handler.isProduceStatusUpdateOnEveryEvent(), - handler.getMaxOutputElementsPerBundle(), - latestContinousSequence) + eventsAndTickers + .apply( + ParDo.of( + new GlobalSequencesProcessorDoFn<>( + handler.getEventExaminer(), + eventCoder, + stateCoder, + keyCoder, + mainOutput, + statusOutput, + handler.getStatusUpdateFrequency(), + unprocessedEventOutput, + handler.isProduceStatusUpdateOnEveryEvent(), + handler.getMaxOutputElementsPerBundle(), + latestContinuousSequence) ) - .withOutputTags( - mainOutput, - TupleTagList.of(Arrays.asList(statusOutput, unprocessedEventOutput))) - .withSideInput(GLOBAL_SEQUENCE_TRACKER, latestContinousSequence) - ); + .withOutputTags( + mainOutput, + TupleTagList.of(Arrays.asList(statusOutput, unprocessedEventOutput))) + .withSideInput(GLOBAL_SEQUENCE_TRACKER, latestContinuousSequence) + ); break; case PER_KEY: @@ -217,4 +219,15 @@ private static Coder getOrderedProcessingStatusCoder(Pi return result; } + static class EventToSequenceAndTimestampConverter + extends DoFn>, SequenceAndTimestamp> { + + @ProcessElement + public void convert(@Element KV> element, + @Timestamp Instant timestamp, OutputReceiver outputReceiver) { + outputReceiver.output( + SequenceAndTimestamp.create(element.getValue().getKey(), timestamp)); + } + } + } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/PerKeyTickerGenerator.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/PerKeyTickerGenerator.java new file mode 100644 index 000000000000..43f278f30dd3 --- /dev/null +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/PerKeyTickerGenerator.java @@ -0,0 +1,110 @@ +package org.apache.beam.sdk.extensions.ordered; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.TimeDomain; +import org.apache.beam.sdk.state.Timer; +import org.apache.beam.sdk.state.TimerSpec; +import org.apache.beam.sdk.state.TimerSpecs; +import org.apache.beam.sdk.state.ValueState; +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.KV; +import org.apache.beam.sdk.values.PCollection; +import org.checkerframework.checker.initialization.qual.Initialized; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.checkerframework.checker.nullness.qual.UnknownKeyFor; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class PerKeyTickerGenerator + extends PTransform>>, + PCollection>>> { + + private static final Logger LOG = LoggerFactory.getLogger(PerKeyTickerGenerator.class); + + private final Coder eventKeyCoder; + private final Coder eventCoder; + private final Duration tickerFrequency; + + PerKeyTickerGenerator(Coder eventKeyCoder, Coder eventCoder, + Duration tickerFrequency) { + this.eventKeyCoder = eventKeyCoder; + this.eventCoder = eventCoder; + this.tickerFrequency = tickerFrequency; + } + + @Override + public @UnknownKeyFor @NonNull @Initialized PCollection>> expand( + PCollection>> input) { + return input.apply("Generate Tickers", + ParDo.of(new PerKeyTickerGeneratorDoFn<>(eventKeyCoder, tickerFrequency))) + .setCoder( + KvCoder.of(eventKeyCoder, KvCoder.of(VarLongCoder.of(), NullableCoder.of(eventCoder)))); + } + + static class PerKeyTickerGeneratorDoFn + extends DoFn>, KV>> { + + private final static String STATE = "state"; + private final static String TIMER = "timer"; + + @StateId(STATE) + @SuppressWarnings("unused") + private final StateSpec> stateSpec; + + @TimerId(TIMER) + @SuppressWarnings("unused") + private final TimerSpec tickerTimer = TimerSpecs.timer(TimeDomain.PROCESSING_TIME); + + private final Duration tickerFrequency; + + + PerKeyTickerGeneratorDoFn(Coder keyCoder, Duration tickerFrequency) { + stateSpec = StateSpecs.value(keyCoder); + this.tickerFrequency = tickerFrequency; + } + + @ProcessElement + public void process( + @Element KV> element, + @StateId(STATE) ValueState state, + @TimerId(TIMER) Timer tickerTimer) { + // Keys are usually simple types. The difference in cost of reading those types from + // state storage of those types comparing to the cost of reading a Boolean should be + // negligible. + @Nullable EventKeyT keyValue = state.read(); + if (keyValue != null) { + return; + } + + tickerTimer.offset(tickerFrequency).setRelative(); + + state.write(element.getKey()); + } + + @OnTimer(TIMER) + public void onTimer( + @StateId(STATE) ValueState state, + @TimerId(TIMER) Timer tickerTimer, + OutputReceiver>> outputReceiver) { + + @Nullable EventKeyT key = state.read(); + if (key == null) { + LOG.error("Expected to get the key from the state, but got null"); + return; + } + + outputReceiver.output(KV.of(key, KV.of(0L, null))); + tickerTimer.offset(tickerFrequency).setRelative(); + } + + } +} From a4fc7f15031208671852925d31bce2e2fbe3fcd8 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Tue, 27 Aug 2024 11:41:43 -0700 Subject: [PATCH 05/33] Corrected the emission batch logic. --- sdks/java/extensions/ordered/build.gradle | 2 +- .../ordered/GlobalSequenceTracker.java | 40 ++++++++++++++++--- .../ordered/GlobalSequencesProcessorDoFn.java | 32 +++++++++------ .../extensions/ordered/ProcessingState.java | 33 ++++++++++++--- .../sdk/extensions/ordered/ProcessorDoFn.java | 19 +++++++-- .../ordered/SequencePerKeyProcessorDoFn.java | 20 ++++------ ...deredEventProcessorGlobalSequenceTest.java | 6 +-- .../src/test/resources/logging.properties | 15 +++++++ 8 files changed, 124 insertions(+), 43 deletions(-) create mode 100644 sdks/java/extensions/ordered/src/test/resources/logging.properties diff --git a/sdks/java/extensions/ordered/build.gradle b/sdks/java/extensions/ordered/build.gradle index dbf88483c50d..338400c03554 100644 --- a/sdks/java/extensions/ordered/build.gradle +++ b/sdks/java/extensions/ordered/build.gradle @@ -28,7 +28,7 @@ dependencies { implementation library.java.vendored_guava_32_1_2_jre testImplementation library.java.junit testImplementation library.java.hamcrest - testImplementation library.java.slf4j_simple + testImplementation library.java.slf4j_jdk14 testImplementation project(path: ':sdks:java:core') testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") } \ No newline at end of file diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java index 342f09755028..4772a2e98cbc 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java @@ -5,7 +5,6 @@ import java.io.InputStream; import java.io.OutputStream; import java.util.ArrayList; -import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -15,6 +14,9 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.coders.InstantCoder; +import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.extensions.ordered.GlobalSequenceTracker.SequenceAndTimestamp; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; @@ -70,6 +72,34 @@ public static SequenceAndTimestamp create(long sequence, Instant timestamp) { } } + static class SequenceAndTimestampCoder extends CustomCoder { + + private static final SequenceAndTimestampCoder INSTANCE = new SequenceAndTimestampCoder(); + + static SequenceAndTimestampCoder of() { + return INSTANCE; + } + + private SequenceAndTimestampCoder() { + } + + @Override + public void encode(SequenceAndTimestamp value, + @UnknownKeyFor @NonNull @Initialized OutputStream outStream) + throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { + VarLongCoder.of().encode(value.getSequence(), outStream); + InstantCoder.of().encode(value.getTimestamp(), outStream); + } + + @Override + public SequenceAndTimestamp decode(@UnknownKeyFor @NonNull @Initialized InputStream inStream) + throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { + long sequence = VarLongCoder.of().decode(inStream); + Instant timestamp = InstantCoder.of().decode(inStream); + return SequenceAndTimestamp.create(sequence, timestamp); + } + } + static class GlobalSequenceCombiner extends CombineFn, SequenceAndTimestamp> { @@ -131,14 +161,14 @@ static class AccumulatorCoder extends Coder> { @Override public void encode(RangeMap value, @UnknownKeyFor @NonNull @Initialized OutputStream outStream) - throws @UnknownKeyFor@NonNull@Initialized CoderException, @UnknownKeyFor@NonNull@Initialized IOException { + throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { } @Override public RangeMap decode( @UnknownKeyFor @NonNull @Initialized InputStream inStream) - throws @UnknownKeyFor@NonNull@Initialized CoderException, @UnknownKeyFor@NonNull@Initialized IOException { + throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { return TreeRangeMap.create(); } @@ -149,7 +179,7 @@ public RangeMap decode( @Override public void verifyDeterministic() - throws @UnknownKeyFor@NonNull@Initialized NonDeterministicException { + throws @UnknownKeyFor @NonNull @Initialized NonDeterministicException { } } @@ -158,7 +188,7 @@ public void verifyDeterministic() public @UnknownKeyFor @NonNull @Initialized Coder> getAccumulatorCoder( @UnknownKeyFor @NonNull @Initialized CoderRegistry registry, @UnknownKeyFor @NonNull @Initialized Coder inputCoder) - throws @UnknownKeyFor@NonNull@Initialized CannotProvideCoderException { + throws @UnknownKeyFor @NonNull @Initialized CannotProvideCoderException { return new AccumulatorCoder(); } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java index 08d72c2eb454..9e8205789edb 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java @@ -18,7 +18,6 @@ import org.apache.beam.sdk.values.TupleTag; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; -import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -118,13 +117,14 @@ public void processElement(ProcessContext context, SequenceAndTimestamp lastContinuousSequence = context.sideInput( latestContinuousSequenceSideInput); - LOG.info("lastSequence: " + lastContinuousSequence); EventT event = eventAndSequence.getValue().getValue(); EventKeyT key = eventAndSequence.getKey(); long sequence = eventAndSequence.getValue().getKey(); + LOG.info(key + ": " + sequence + " lastSequence: " + lastContinuousSequence); + ProcessingState processingState = processingStateState.read(); if (processingState == null) { @@ -141,6 +141,12 @@ public void processElement(ProcessContext context, if (event == null) { // This is the ticker event. We only need to update the state as it relates to the global sequence. processingStateState.write(processingState); + + // TODO: we can keep resetting this into the future under heavy load. + // Need to add logic to avoid doing it. + // + setBatchEmissionTimerIfNeeded(batchEmissionTimer, processingState); + return; } @@ -168,15 +174,18 @@ public void processElement(ProcessContext context, outputReceiver, window.maxTimestamp()); - // TODO: we can keep resetting this into the future under heavy load. - // Need to add logic to avoid doing it. - // - batchEmissionTimer.set(lastContinuousSequence.getTimestamp()); - // Only if the record matches the sequence it can be output now // TODO: refactor the code from SequencePerKeyDoFn } + private void setBatchEmissionTimerIfNeeded(Timer batchEmissionTimer, + ProcessingState processingState) { + SequenceAndTimestamp lastCompleteGlobalSequence = processingState.getLastCompleteGlobalSequence(); + if (processingState.getBufferedEventCount() > 0 && lastCompleteGlobalSequence != null) { + batchEmissionTimer.set(lastCompleteGlobalSequence.getTimestamp()); + } + } + @OnTimer(BATCH_EMISSION_TIMER) public void onBatchEmission( OnTimerContext context, @@ -196,7 +205,7 @@ public void onBatchEmission( StateT state = mutableStateState.read(); - Long lastCompleteGlobalSequence = processingState.getLastCompleteGlobalSequence(); + SequenceAndTimestamp lastCompleteGlobalSequence = processingState.getLastCompleteGlobalSequence(); if (lastCompleteGlobalSequence == null) { LOG.warn("Last complete global instance is null."); return; @@ -207,12 +216,11 @@ public void onBatchEmission( LOG.warn("Earliest buffered sequence is null."); return; } - Instant startRange = fromLong(earliestBufferedSequence); - Instant endRange = fromLong(lastCompleteGlobalSequence + 1); - LOG.info("Processing range " + processingState.getKey() + " " + startRange + " " + endRange); + LOG.info("Emission timer: " + processingState); + state = processBufferedEventRange(processingState, state, bufferedEventsState, outputReceiver, - batchEmissionTimer, startRange, endRange); + batchEmissionTimer, lastCompleteGlobalSequence.getSequence()); saveStates( processingStatusState, diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java index 8bc5488c0efc..6c0f3df9d346 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java @@ -29,6 +29,8 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.extensions.ordered.GlobalSequenceTracker.SequenceAndTimestamp; +import org.apache.beam.sdk.extensions.ordered.GlobalSequenceTracker.SequenceAndTimestampCoder; +import org.apache.beam.sdk.schemas.SchemaCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.initialization.qual.Initialized; @@ -52,7 +54,7 @@ class ProcessingState { private long resultCount; - @Nullable private Long lastCompleteGlobalSequence; + @Nullable private SequenceAndTimestamp lastCompleteGlobalSequence; private KeyT key; @@ -134,11 +136,11 @@ public KeyT getKey() { return key; } - public @Nullable Long getLastCompleteGlobalSequence() { + public @Nullable SequenceAndTimestamp getLastCompleteGlobalSequence() { return lastCompleteGlobalSequence; } - public void setLastCompleteGlobalSequence(@Nullable Long lastCompleteGlobalSequence) { + public void setLastCompleteGlobalSequence(@Nullable SequenceAndTimestamp lastCompleteGlobalSequence) { this.lastCompleteGlobalSequence = lastCompleteGlobalSequence; } @@ -241,6 +243,22 @@ public int hashCode() { key); } + @Override + public String toString() { + return "ProcessingState{" + + "lastOutputSequence=" + lastOutputSequence + + ", latestBufferedSequence=" + latestBufferedSequence + + ", earliestBufferedSequence=" + earliestBufferedSequence + + ", bufferedEventCount=" + bufferedEventCount + + ", lastEventReceived=" + lastEventReceived + + ", eventsReceived=" + eventsReceived + + ", duplicates=" + duplicates + + ", resultCount=" + resultCount + + ", lastCompleteGlobalSequence=" + lastCompleteGlobalSequence + + ", key=" + key + + '}'; + } + public boolean isProcessingCompleted() { return lastEventReceived && bufferedEventCount == 0; } @@ -288,7 +306,7 @@ public long resultsProducedInBundle(long numberOfResultsBeforeBundleStart) { public void updateGlobalSequenceDetails(SequenceAndTimestamp updated) { // TODO: do we need to select max? Do we care about the timestamp? - this.lastCompleteGlobalSequence = updated.getSequence(); + this.lastCompleteGlobalSequence = updated; } /** @@ -304,6 +322,9 @@ static class ProcessingStateCoder extends Coder> { private static final VarIntCoder INTEGER_CODER = VarIntCoder.of(); private static final BooleanCoder BOOLEAN_CODER = BooleanCoder.of(); + private static final NullableCoder SEQUENCE_AND_TIMESTAMP_CODER = + NullableCoder.of(SequenceAndTimestampCoder.of()); + private Coder keyCoder; private ProcessingStateCoder(Coder keyCoder) { @@ -325,7 +346,7 @@ public void encode(ProcessingState value, OutputStream outStream) throws I LONG_CODER.encode(value.getResultCount(), outStream); BOOLEAN_CODER.encode(value.isLastEventReceived(), outStream); keyCoder.encode(value.getKey(), outStream); - NULLABLE_LONG_CODER.encode(value.getLastCompleteGlobalSequence(), outStream); + SEQUENCE_AND_TIMESTAMP_CODER.encode(value.getLastCompleteGlobalSequence(), outStream); } @Override @@ -339,7 +360,7 @@ public ProcessingState decode(InputStream inStream) throws IOException { long resultCount = LONG_CODER.decode(inStream); boolean isLastEventReceived = BOOLEAN_CODER.decode(inStream); KeyT key = keyCoder.decode(inStream); - Long lastCompleteGlobalSequence = NULLABLE_LONG_CODER.decode(inStream); + SequenceAndTimestamp lastCompleteGlobalSequence = SEQUENCE_AND_TIMESTAMP_CODER.decode(inStream); ProcessingState result = new ProcessingState<>( key, diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java index b00b8e7b8c9c..9ead8b611b5c 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java @@ -242,6 +242,7 @@ protected void emitProcessingStatus( ProcessingState processingState, MultiOutputReceiver outputReceiver, Instant statusTimestamp) { + LOG.info("Emitting status for: " + processingState.getKey() + ", " + processingState); outputReceiver .get(statusTupleTag) .outputWithTimestamp( @@ -293,7 +294,15 @@ private void bufferEvent(long currentSequence, EventTypeT currentEvent, @Nullable StateTypeT processBufferedEventRange(ProcessingState processingState, @Nullable StateTypeT state, OrderedListState bufferedEventsState, MultiOutputReceiver outputReceiver, - Timer largeBatchEmissionTimer, Instant startRange, Instant endRange) { + Timer largeBatchEmissionTimer, long lastKnownCompleteSequenceNumber) { + Long earliestBufferedSequence = processingState.getEarliestBufferedSequence(); + Long latestBufferedSequence = processingState.getLatestBufferedSequence(); + if(earliestBufferedSequence == null || latestBufferedSequence == null) { + return state; + } + Instant startRange = fromLong(earliestBufferedSequence); + Instant endRange = fromLong(latestBufferedSequence + 1); + // readRange is efficiently implemented and will bring records in batches Iterable> events = bufferedEventsState.readRange(startRange, endRange); @@ -320,9 +329,11 @@ private void bufferEvent(long currentSequence, EventTypeT currentEvent, } Long lastOutputSequence = processingState.getLastOutputSequence(); - if (checkForSequenceGapInBufferedEvents() && - lastOutputSequence != null && - eventSequence > lastOutputSequence + 1) { + boolean currentEventIsNotInSequence = lastOutputSequence != null && eventSequence > lastOutputSequence + 1; + boolean breakOutput = checkForSequenceGapInBufferedEvents() ? + currentEventIsNotInSequence : + (eventSequence > lastKnownCompleteSequenceNumber || currentEventIsNotInSequence); + if (breakOutput) { processingState.foundSequenceGap(eventSequence); // Records will be cleared up to this element endClearRange = fromLong(eventSequence); diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java index 26676370d954..077fade4417e 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java @@ -1,11 +1,9 @@ package org.apache.beam.sdk.extensions.ordered; -import java.util.Iterator; import javax.annotation.Nullable; import org.apache.beam.sdk.coders.BooleanCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.extensions.ordered.ProcessingState.ProcessingStateCoder; -import org.apache.beam.sdk.extensions.ordered.UnprocessedEvent.Reason; import org.apache.beam.sdk.state.OrderedListState; import org.apache.beam.sdk.state.StateSpec; import org.apache.beam.sdk.state.StateSpecs; @@ -16,16 +14,15 @@ import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; import org.joda.time.Duration; -import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; class SequencePerKeyProcessorDoFn> extends ProcessorDoFn { + private static final Logger LOG = LoggerFactory.getLogger(SequencePerKeyProcessorDoFn.class); private static final String LARGE_BATCH_EMISSION_TIMER = "largeBatchTimer"; @@ -50,6 +47,7 @@ class SequencePerKeyProcessorDoFn>> processingStateSpec; + /** * Stateful DoFn to do the bulk of processing. * @@ -186,19 +184,16 @@ private void processBufferedEvents( // Technically this block is not needed because these preconditions are checked // earlier. Included to keep the linter happy. Long earliestBufferedSequence = processingState.getEarliestBufferedSequence(); - if(earliestBufferedSequence == null) { + if (earliestBufferedSequence == null) { return; } Long latestBufferedSequence = processingState.getLatestBufferedSequence(); - if(latestBufferedSequence == null) { + if (latestBufferedSequence == null) { return; } - Instant startRange = fromLong(earliestBufferedSequence); - Instant endRange = fromLong(latestBufferedSequence + 1); - - processBufferedEventRange(processingState, state, bufferedEventsState, outputReceiver, largeBatchEmissionTimer, - startRange, endRange); + processBufferedEventRange(processingState, state, bufferedEventsState, outputReceiver, + largeBatchEmissionTimer, Long.MIN_VALUE); } @OnTimer(LARGE_BATCH_EMISSION_TIMER) @@ -249,7 +244,8 @@ public void onStatusEmission( @StateId(PROCESSING_STATE) ValueState> processingStateState) { - processStatusTimerEvent(outputReceiver, statusEmissionTimer, windowClosedState, processingStateState); + processStatusTimerEvent(outputReceiver, statusEmissionTimer, windowClosedState, + processingStateState); } @OnWindowExpiration diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java index 9a0d3fc8376a..6a06911ca20b 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java @@ -42,7 +42,7 @@ public void testPerfectOrderingProcessing() throws CannotProvideCoderException { 0, null, null, - 4, + Arrays.stream(events).filter(e -> e.getKey().equals("id-1")).count(), Arrays.stream(events).filter(e -> e.getKey().equals("id-1")).count(), 0, false))); @@ -50,11 +50,11 @@ public void testPerfectOrderingProcessing() throws CannotProvideCoderException { KV.of( "id-2", OrderedProcessingStatus.create( - 1L, + 5L, 0, null, null, - 2, + Arrays.stream(events).filter(e -> e.getKey().equals("id-2")).count(), Arrays.stream(events).filter(e -> e.getKey().equals("id-2")).count(), 0, false))); diff --git a/sdks/java/extensions/ordered/src/test/resources/logging.properties b/sdks/java/extensions/ordered/src/test/resources/logging.properties new file mode 100644 index 000000000000..c324d864a083 --- /dev/null +++ b/sdks/java/extensions/ordered/src/test/resources/logging.properties @@ -0,0 +1,15 @@ +handlers = java.util.logging.ConsoleHandler +java.util.logging.ConsoleHandler.level = ALL +java.util.logging.ConsoleHandler.formatter = java.util.logging.SimpleFormatter +# Pattern works since Java 7 +java.util.logging.SimpleFormatter.format = [%1$tc] %4$s: %2$s - %5$s %6$s%n + +# Configure logging levels +# Available log levels are: +# OFF, SEVERE, WARNING, INFO, CONFIG, FINE, FINER, FINEST, ALL + +# root logger +.level = INFO + +# child logger +org.example.level = ALL \ No newline at end of file From 59e442652bfb130e3db3d54c5e4b609e9cce34df Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Tue, 27 Aug 2024 15:54:08 -0700 Subject: [PATCH 06/33] Reworked some tests and fixed the batch output logic. --- .../sdk/extensions/ordered/ProcessorDoFn.java | 6 +- ...deredEventProcessorGlobalSequenceTest.java | 304 ++---------------- .../ordered/OrderedEventProcessorTest.java | 5 +- 3 files changed, 41 insertions(+), 274 deletions(-) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java index 9ead8b611b5c..76a2097ec006 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java @@ -330,10 +330,10 @@ private void bufferEvent(long currentSequence, EventTypeT currentEvent, Long lastOutputSequence = processingState.getLastOutputSequence(); boolean currentEventIsNotInSequence = lastOutputSequence != null && eventSequence > lastOutputSequence + 1; - boolean breakOutput = checkForSequenceGapInBufferedEvents() ? + boolean stopProcessing = checkForSequenceGapInBufferedEvents() ? currentEventIsNotInSequence : - (eventSequence > lastKnownCompleteSequenceNumber || currentEventIsNotInSequence); - if (breakOutput) { + (! (eventSequence <= lastKnownCompleteSequenceNumber) && currentEventIsNotInSequence); + if (stopProcessing) { processingState.foundSequenceGap(eventSequence); // Records will be cleared up to this element endClearRange = fromLong(eventSequence); diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java index 6a06911ca20b..9135636f30db 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java @@ -1,7 +1,6 @@ package org.apache.beam.sdk.extensions.ordered; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.List; import org.apache.beam.sdk.Pipeline; @@ -33,32 +32,6 @@ public void testPerfectOrderingProcessing() throws CannotProvideCoderException { Event.create(5, "id-2", "b") }; - Collection> expectedStatuses = new ArrayList<>(); - expectedStatuses.add( - KV.of( - "id-1", - OrderedProcessingStatus.create( - 3L, - 0, - null, - null, - Arrays.stream(events).filter(e -> e.getKey().equals("id-1")).count(), - Arrays.stream(events).filter(e -> e.getKey().equals("id-1")).count(), - 0, - false))); - expectedStatuses.add( - KV.of( - "id-2", - OrderedProcessingStatus.create( - 5L, - 0, - null, - null, - Arrays.stream(events).filter(e -> e.getKey().equals("id-2")).count(), - Arrays.stream(events).filter(e -> e.getKey().equals("id-2")).count(), - 0, - false))); - Collection> expectedOutput = new ArrayList<>(); expectedOutput.add(KV.of("id-1", "a")); expectedOutput.add(KV.of("id-1", "ab")); @@ -69,12 +42,10 @@ public void testPerfectOrderingProcessing() throws CannotProvideCoderException { testGlobalSequenceProcessing( events, - expectedStatuses, expectedOutput, EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT, - DONT_PRODUCE_STATUS_ON_EVERY_EVENT); + LARGE_MAX_RESULTS_PER_OUTPUT); } @Test @@ -84,39 +55,13 @@ public void testOutOfSequenceProcessing() throws CannotProvideCoderException { Event.create(1, "id-1", "b"), Event.create(0, "id-1", "a"), Event.create(3, "id-1", "d"), - Event.create(1, "id-2", "b"), - Event.create(2, "id-2", "c"), - Event.create(4, "id-2", "e"), - Event.create(0, "id-2", "a"), - Event.create(3, "id-2", "d") + Event.create(5, "id-2", "b"), + Event.create(6, "id-2", "c"), + Event.create(8, "id-2", "e"), + Event.create(4, "id-2", "a"), + Event.create(7, "id-2", "d") }; - Collection> expectedStatuses = new ArrayList<>(); - expectedStatuses.add( - KV.of( - "id-1", - OrderedProcessingStatus.create( - 3L, - 0, - null, - null, - 4, - Arrays.stream(events).filter(e -> e.getKey().equals("id-1")).count(), - 0, - false))); - expectedStatuses.add( - KV.of( - "id-2", - OrderedProcessingStatus.create( - 4L, - 0, - null, - null, - 5, - Arrays.stream(events).filter(e -> e.getKey().equals("id-2")).count(), - 0, - false))); - Collection> expectedOutput = new ArrayList<>(); expectedOutput.add(KV.of("id-1", "a")); expectedOutput.add(KV.of("id-1", "ab")); @@ -130,37 +75,10 @@ public void testOutOfSequenceProcessing() throws CannotProvideCoderException { testGlobalSequenceProcessing( events, - expectedStatuses, expectedOutput, EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT, - DONT_PRODUCE_STATUS_ON_EVERY_EVENT); - } - - @Test - public void testUnfinishedProcessing() throws CannotProvideCoderException { - Event[] events = { - Event.create(2, "id-1", "c"), - // Excluded Event.create(1, "id-1", "b"), - Event.create(0, "id-1", "a"), - Event.create(3, "id-1", "d"), - Event.create(0, "id-2", "a"), - Event.create(1, "id-2", "b"), - }; - - Collection> expectedStatuses = new ArrayList<>(); - expectedStatuses.add( - KV.of("id-1", OrderedProcessingStatus.create(0L, 2, 2L, 3L, 3, 1L, 0, false))); - expectedStatuses.add( - KV.of("id-2", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2L, 0, false))); - - Collection> expectedOutput = new ArrayList<>(); - expectedOutput.add(KV.of("id-1", "a")); - expectedOutput.add(KV.of("id-2", "a")); - expectedOutput.add(KV.of("id-2", "ab")); - - testGlobalSequenceProcessing(events, expectedStatuses, expectedOutput, 1, 0, 1000, false); + LARGE_MAX_RESULTS_PER_OUTPUT); } @Test @@ -168,25 +86,18 @@ public void testHandlingOfDuplicateSequences() throws CannotProvideCoderExceptio Event[] events = { Event.create(3, "id-1", "d"), Event.create(2, "id-1", "c"), - // Duplicates to be buffered + + // Duplicates Event.create(3, "id-1", "d"), Event.create(3, "id-1", "d"), + Event.create(0, "id-1", "a"), Event.create(1, "id-1", "b"), - // Duplicates after the events are processed + // Additional duplicates Event.create(1, "id-1", "b"), Event.create(3, "id-1", "d"), }; - int resultCount = 4; - int duplicateCount = 4; - - Collection> expectedStatuses = new ArrayList<>(); - expectedStatuses.add( - KV.of( - "id-1", - OrderedProcessingStatus.create( - 3L, 0, null, null, events.length, resultCount, duplicateCount, false))); Collection> expectedOutput = new ArrayList<>(); expectedOutput.add(KV.of("id-1", "a")); @@ -202,13 +113,11 @@ public void testHandlingOfDuplicateSequences() throws CannotProvideCoderExceptio testGlobalSequenceProcessing( events, - expectedStatuses, expectedOutput, duplicates, EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT, - DONT_PRODUCE_STATUS_ON_EVERY_EVENT); + LARGE_MAX_RESULTS_PER_OUTPUT); } @Test @@ -220,13 +129,13 @@ public void testHandlingOfCheckedExceptions() throws CannotProvideCoderException Event.create(3, "id-1", "c"), }; - Collection> expectedStatuses = new ArrayList<>(); - expectedStatuses.add( - KV.of("id-1", OrderedProcessingStatus.create(1L, 1, 3L, 3L, events.length, 2, 0, false))); - + // This is an interesting case - even though event #2 is not processed it doesn't affect + // the global sequence calculations. It is not considered a gap, and all the subsequent + // events will be processed. Collection> expectedOutput = new ArrayList<>(); expectedOutput.add(KV.of("id-1", "a")); expectedOutput.add(KV.of("id-1", "ab")); + expectedOutput.add(KV.of("id-1", "abc")); Collection>>> failedEvents = new ArrayList<>(); failedEvents.add( @@ -238,13 +147,11 @@ public void testHandlingOfCheckedExceptions() throws CannotProvideCoderException testGlobalSequenceProcessing( events, - expectedStatuses, expectedOutput, failedEvents, EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT, - DONT_PRODUCE_STATUS_ON_EVERY_EVENT); + LARGE_MAX_RESULTS_PER_OUTPUT); } @Test @@ -254,16 +161,10 @@ public void testProcessingWithEveryOtherResultEmission() throws CannotProvideCod Event.create(1, "id-1", "b"), Event.create(0, "id-1", "a"), Event.create(3, "id-1", "d"), - Event.create(0, "id-2", "a"), - Event.create(1, "id-2", "b"), + Event.create(4, "id-2", "a"), + Event.create(5, "id-2", "b"), }; - Collection> expectedStatuses = new ArrayList<>(); - expectedStatuses.add( - KV.of("id-1", OrderedProcessingStatus.create(3L, 0, null, null, 4, 2L, 0, false))); - expectedStatuses.add( - KV.of("id-2", OrderedProcessingStatus.create(1L, 0, null, null, 2, 1L, 0, false))); - Collection> expectedOutput = new ArrayList<>(); expectedOutput.add(KV.of("id-1", "a")); // Skipped KV.of("id-1", "ab"), @@ -273,12 +174,10 @@ public void testProcessingWithEveryOtherResultEmission() throws CannotProvideCod // Skipped KV.of("id-2", "ab") testGlobalSequenceProcessing( events, - expectedStatuses, expectedOutput, EMISSION_FREQUENCY_ON_EVERY_OTHER_EVENT, INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT, - DONT_PRODUCE_STATUS_ON_EVERY_EVENT); + LARGE_MAX_RESULTS_PER_OUTPUT); } @Test @@ -295,72 +194,24 @@ public void testLargeBufferedOutputInTimer() throws CannotProvideCoderException List events = new ArrayList<>(sequences.length); Collection> expectedOutput = new ArrayList<>(sequences.length); - Collection> expectedStatuses = - new ArrayList<>(sequences.length + 10); StringBuilder output = new StringBuilder(); String outputPerElement = "."; String key = "id-1"; - int bufferedEventCount = 0; - for (long sequence : sequences) { - ++bufferedEventCount; - events.add(Event.create(sequence, key, outputPerElement)); output.append(outputPerElement); expectedOutput.add(KV.of(key, output.toString())); - if (bufferedEventCount < sequences.length) { - // Last event will result in a batch of events being produced. That's why it's excluded - // here. - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - null, bufferedEventCount, 2L, sequence, bufferedEventCount, 0L, 0, false))); - } - } - - // Statuses produced by the batched processing - for (int i = maxResultsPerOutput; i < sequences.length; i += maxResultsPerOutput) { - long lastOutputSequence = i; - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - lastOutputSequence, - sequences.length - lastOutputSequence, - lastOutputSequence + 1, - (long) sequences.length, - sequences.length, - lastOutputSequence, - 0, - false))); } - // -- Final status - indicates that everything has been fully processed - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - (long) sequences.length, - 0, - null, - null, - sequences.length, - sequences.length, - 0, - false))); - testGlobalSequenceProcessing( events.toArray(new Event[events.size()]), - expectedStatuses, expectedOutput, EMISSION_FREQUENCY_ON_EVERY_ELEMENT, 1L /* This dataset assumes 1 as the starting sequence */, - maxResultsPerOutput, - PRODUCE_STATUS_ON_EVERY_EVENT); + maxResultsPerOutput); } @Test @@ -382,80 +233,12 @@ public void testSequenceGapProcessingInBufferedOutput() throws CannotProvideCode expectedOutput.add(KV.of(key, output.toString())); } - int numberOfReceivedEvents = 0; - Collection> expectedStatuses = new ArrayList<>(); - - // First elements are out-of-sequence and they just get buffered. Earliest and latest sequence - // numbers keep changing. - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - null, 1, 2L, 2L, ++numberOfReceivedEvents, 0L, 0, false))); - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - null, 2, 2L, 3L, ++numberOfReceivedEvents, 0L, 0, false))); - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - null, 3, 2L, 7L, ++numberOfReceivedEvents, 0L, 0, false))); - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - null, 4, 2L, 8L, ++numberOfReceivedEvents, 0L, 0, false))); - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - null, 5, 2L, 9L, ++numberOfReceivedEvents, 0L, 0, false))); - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - null, 6, 2L, 10L, ++numberOfReceivedEvents, 0L, 0, false))); - // --- 1 has appeared and caused the batch to be sent out. - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - 3L, 4, 7L, 10L, ++numberOfReceivedEvents, 3L, 0, false))); - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - 4L, 4, 7L, 10L, ++numberOfReceivedEvents, 4L, 0, false))); - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - 5L, 4, 7L, 10L, ++numberOfReceivedEvents, 5L, 0, false))); - // --- 6 came and 6, 7, and 8 got output - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - 8L, 2, 9L, 10L, ++numberOfReceivedEvents, 8L, 0, false))); - // Last timer run produces the final status. Number of received events doesn't - // increase, - // this is the result of a timer processing - expectedStatuses.add( - KV.of( - key, - OrderedProcessingStatus.create( - 10L, 0, null, null, numberOfReceivedEvents, 10L, 0, false))); - testGlobalSequenceProcessing( events.toArray(new Event[events.size()]), - expectedStatuses, expectedOutput, EMISSION_FREQUENCY_ON_EVERY_ELEMENT, 1L /* This dataset assumes 1 as the starting sequence */, - maxResultsPerOutput, - PRODUCE_STATUS_ON_EVERY_EVENT); + maxResultsPerOutput); } @Test @@ -466,10 +249,6 @@ public void testHandlingOfMaxSequenceNumber() throws CannotProvideCoderException Event.create(Long.MAX_VALUE, "id-1", "c") }; - Collection> expectedStatuses = new ArrayList<>(); - expectedStatuses.add( - KV.of("id-1", OrderedProcessingStatus.create(1L, 0, null, null, 3, 2, 0, false))); - Collection> expectedOutput = new ArrayList<>(); expectedOutput.add(KV.of("id-1", "a")); expectedOutput.add(KV.of("id-1", "ab")); @@ -485,13 +264,11 @@ public void testHandlingOfMaxSequenceNumber() throws CannotProvideCoderException testGlobalSequenceProcessing( events, - expectedStatuses, expectedOutput, unprocessedEvents, EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT, - DONT_PRODUCE_STATUS_ON_EVERY_EVENT); + LARGE_MAX_RESULTS_PER_OUTPUT); } @Test @@ -502,13 +279,6 @@ public void testProcessingOfTheLastInput() throws CannotProvideCoderException { Event.create(2, "id-1", StringEventExaminer.LAST_INPUT) }; - Collection> expectedStatuses = new ArrayList<>(); - expectedStatuses.add( - KV.of( - "id-1", - OrderedProcessingStatus.create( - 2L, 0, null, null, events.length, events.length, 0, LAST_EVENT_RECEIVED))); - Collection> expectedOutput = new ArrayList<>(); expectedOutput.add(KV.of("id-1", "a")); expectedOutput.add(KV.of("id-1", "ab")); @@ -516,68 +286,64 @@ public void testProcessingOfTheLastInput() throws CannotProvideCoderException { testGlobalSequenceProcessing( events, - expectedStatuses, expectedOutput, EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT, - DONT_PRODUCE_STATUS_ON_EVERY_EVENT); + LARGE_MAX_RESULTS_PER_OUTPUT); } private void testGlobalSequenceProcessing( Event[] events, - Collection> expectedStatuses, Collection> expectedOutput, int emissionFrequency, long initialSequence, - int maxResultsPerOutput, - boolean produceStatusOnEveryEvent) + int maxResultsPerOutput) throws CannotProvideCoderException { testGlobalSequenceProcessing( events, - expectedStatuses, expectedOutput, NO_EXPECTED_DLQ_EVENTS, emissionFrequency, initialSequence, - maxResultsPerOutput, - produceStatusOnEveryEvent); + maxResultsPerOutput); } private void testGlobalSequenceProcessing( Event[] events, - Collection> expectedStatuses, Collection> expectedOutput, Collection>>> expectedUnprocessedEvents, int emissionFrequency, long initialSequence, - int maxResultsPerOutput, - boolean produceStatusOnEveryEvent) + int maxResultsPerOutput) throws CannotProvideCoderException { // Test a streaming pipeline doTest( events, - expectedStatuses, + null /* expectedStatuses */, expectedOutput, expectedUnprocessedEvents, emissionFrequency, initialSequence, maxResultsPerOutput, - produceStatusOnEveryEvent, + false /* produceStatusOnEveryEvent */, STREAMING, SequenceType.GLOBAL); + if (true) { + // TODO: Test batch processing + return; + } // Test a batch pipeline doTest( events, - expectedStatuses, + null /* expectedStatuses */, expectedOutput, expectedUnprocessedEvents, emissionFrequency, initialSequence, maxResultsPerOutput, - produceStatusOnEveryEvent, + false /* produceStatusOnEveryEvent */, BATCH, SequenceType.GLOBAL); } diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTest.java index 1830b0f8a225..d140b8f4c6cd 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTest.java @@ -21,6 +21,7 @@ import java.util.Collection; import java.util.Collections; import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.extensions.ordered.UnprocessedEvent.Reason; @@ -109,7 +110,7 @@ public void map( */ protected void doTest( Event[] events, - Collection> expectedStatuses, + @Nullable Collection> expectedStatuses, Collection> expectedOutput, Collection>>> expectedUnprocessedEvents, int emissionFrequency, @@ -151,7 +152,7 @@ protected void doTest( PAssert.that("Output matches", processingResult.output()).containsInAnyOrder(expectedOutput); - if (streaming) { + if (streaming && expectedStatuses != null) { // Only in a streaming pipeline the events will arrive in a pre-determined order and the statuses // will be deterministic. In batch pipelines events can be processed in any order, // so we skip status verification and rely on the output and unprocessed event matches. From ea6209dfd77e3bf5ed814ed278ba05174504695b Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Fri, 30 Aug 2024 12:42:20 -0700 Subject: [PATCH 07/33] Pluggable combiner for the global sequence. --- .../ordered/CompletedSequenceRange.java | 26 +++ .../ordered/GlobalSequenceTracker.java | 172 ++++-------------- .../ordered/GlobalSequencesProcessorDoFn.java | 15 +- .../ordered/OrderedEventProcessor.java | 19 +- .../ordered/OrderedProcessingHandler.java | 10 + .../extensions/ordered/ProcessingState.java | 14 +- .../sdk/extensions/ordered/ProcessorDoFn.java | 5 +- .../ordered/SequencePerKeyProcessorDoFn.java | 2 +- .../extensions/ordered/UnprocessedEvent.java | 1 + .../combiner/DefaultSequenceCombiner.java | 77 ++++++++ .../combiner/SequenceRangeAccumulator.java | 78 ++++++++ .../StringBufferOrderedProcessingHandler.java | 2 + .../SequenceRangeAccumulatorTest.java | 46 +++++ 13 files changed, 301 insertions(+), 166 deletions(-) create mode 100644 sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/CompletedSequenceRange.java create mode 100644 sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java create mode 100644 sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java create mode 100644 sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/CompletedSequenceRange.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/CompletedSequenceRange.java new file mode 100644 index 000000000000..230e5b920fbd --- /dev/null +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/CompletedSequenceRange.java @@ -0,0 +1,26 @@ +package org.apache.beam.sdk.extensions.ordered; + +import com.google.auto.value.AutoValue; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.joda.time.Instant; + +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class CompletedSequenceRange { + public static final CompletedSequenceRange EMPTY = + CompletedSequenceRange.create(Long.MIN_VALUE, Long.MIN_VALUE, Instant.ofEpochMilli( + Long.MIN_VALUE)); + + public abstract long getStart(); + public abstract long getEnd(); + public abstract Instant getTimestamp(); + + public boolean isEmpty() { + return this.equals(EMPTY); + } + + public static CompletedSequenceRange create(long start, long end, Instant timestamp) { + return new AutoValue_CompletedSequenceRange(start, end, timestamp); + } +} diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java index 4772a2e98cbc..2601203cf794 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java @@ -1,27 +1,15 @@ package org.apache.beam.sdk.extensions.ordered; -import com.google.auto.value.AutoValue; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.function.BiFunction; -import org.apache.beam.sdk.coders.CannotProvideCoderException; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.InstantCoder; import org.apache.beam.sdk.coders.VarLongCoder; -import org.apache.beam.sdk.extensions.ordered.GlobalSequenceTracker.SequenceAndTimestamp; -import org.apache.beam.sdk.schemas.AutoValueSchema; -import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.extensions.ordered.combiner.DefaultSequenceCombiner; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.AfterFirst; import org.apache.beam.sdk.transforms.windowing.AfterPane; @@ -29,11 +17,11 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Repeatedly; import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.Preconditions; +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.vendor.guava.v32_1_2_jre.com.google.common.collect.Range; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.RangeMap; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.TreeRangeMap; +import org.apache.beam.sdk.values.TimestampedValue; import org.checkerframework.checker.initialization.qual.Initialized; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; @@ -41,38 +29,34 @@ import org.joda.time.Duration; import org.joda.time.Instant; -class GlobalSequenceTracker extends - PTransform, PCollectionView> { +class GlobalSequenceTracker> extends + PTransform>>>, PCollectionView> { + + private final DefaultSequenceCombiner sequenceCombiner; + + public GlobalSequenceTracker(EventExaminer eventExaminer) { + this.sequenceCombiner = new DefaultSequenceCombiner<>(eventExaminer); + } @Override - public PCollectionView expand(PCollection input) { + public PCollectionView expand( + PCollection>>> input) { return input - .apply("Setup Triggering", Window.into(new GlobalWindows()) - .accumulatingFiredPanes() - .triggering( - Repeatedly.forever(AfterFirst.of( - AfterPane.elementCountAtLeast(1), - AfterProcessingTime.pastFirstElementInPane() - .plusDelayOf(Duration.standardSeconds(5)))))) + // TODO: get the windowing strategy from the input rather than assume global windows. + .apply("Setup Triggering", + Window.>>>into(new GlobalWindows()) + .accumulatingFiredPanes() + .triggering( + Repeatedly.forever(AfterFirst.of( + AfterPane.elementCountAtLeast(1), + AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(Duration.standardSeconds(5)))))) .apply("Create Side Input", - Combine.globally(new GlobalSequenceCombiner()).asSingletonView()); - } - - @DefaultSchema(AutoValueSchema.class) - @AutoValue - static abstract class SequenceAndTimestamp { - - public abstract long getSequence(); - - public abstract Instant getTimestamp(); - - public static SequenceAndTimestamp create(long sequence, Instant timestamp) { - return new AutoValue_GlobalSequenceTracker_SequenceAndTimestamp(sequence, timestamp); - } + Combine.globally(sequenceCombiner).asSingletonView()); } - static class SequenceAndTimestampCoder extends CustomCoder { + static class SequenceAndTimestampCoder extends CustomCoder { private static final SequenceAndTimestampCoder INSTANCE = new SequenceAndTimestampCoder(); @@ -84,112 +68,22 @@ private SequenceAndTimestampCoder() { } @Override - public void encode(SequenceAndTimestamp value, + public void encode(CompletedSequenceRange value, @UnknownKeyFor @NonNull @Initialized OutputStream outStream) throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { - VarLongCoder.of().encode(value.getSequence(), outStream); + VarLongCoder.of().encode(value.getStart(), outStream); + VarLongCoder.of().encode(value.getEnd(), outStream); InstantCoder.of().encode(value.getTimestamp(), outStream); } @Override - public SequenceAndTimestamp decode(@UnknownKeyFor @NonNull @Initialized InputStream inStream) + public CompletedSequenceRange decode(@UnknownKeyFor @NonNull @Initialized InputStream inStream) throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { - long sequence = VarLongCoder.of().decode(inStream); + long start = VarLongCoder.of().decode(inStream); + long end = VarLongCoder.of().decode(inStream); Instant timestamp = InstantCoder.of().decode(inStream); - return SequenceAndTimestamp.create(sequence, timestamp); + return CompletedSequenceRange.create(start, end, timestamp); } } - static class GlobalSequenceCombiner extends - CombineFn, SequenceAndTimestamp> { - - public static final BiFunction<@NonNull Instant, @Nullable Instant, @Nullable Instant> OLDEST_TIMESTAMP_SELECTOR = (instant1, instant2) -> { - if (instant2 == null) { - return instant1; - } - @NonNull Instant nonNullableSecondValue = instant2; - return instant1.isAfter(nonNullableSecondValue) ? instant1 : nonNullableSecondValue; - }; - - - @Override - public RangeMap createAccumulator() { - return TreeRangeMap.create(); - } - - @Override - public RangeMap addInput(RangeMap accum, - SequenceAndTimestamp event) { - accum.merge(Range.singleton(event.getSequence()), event.getTimestamp(), - OLDEST_TIMESTAMP_SELECTOR); - return accum; - } - - @Override - public RangeMap mergeAccumulators( - Iterable> accumulators) { - RangeMap newAccum = createAccumulator(); - for (RangeMap accum : accumulators) { - for (Map.Entry, Instant> entry : accum.asMapOfRanges().entrySet()) { - newAccum.merge(entry.getKey(), entry.getValue(), OLDEST_TIMESTAMP_SELECTOR); - } - } - return newAccum; - } - - @Override - public SequenceAndTimestamp extractOutput(RangeMap accum) { - SequenceAndTimestamp output = SequenceAndTimestamp.create(Long.MIN_VALUE, Instant.EPOCH); - Iterator, Instant>> iter = accum.asMapOfRanges().entrySet().iterator(); - Map.Entry, Instant> prevEntry = null; - while (iter.hasNext()) { - Map.Entry, Instant> entry = iter.next(); - if (prevEntry != null - && entry.getKey().lowerEndpoint() > prevEntry.getKey().upperEndpoint() + 1) { - // We have a hole! Break out. - break; - } - output = SequenceAndTimestamp.create(entry.getKey().upperEndpoint(), entry.getValue()); - prevEntry = entry; - } - return output; - } - - - static class AccumulatorCoder extends Coder> { - - @Override - public void encode(RangeMap value, - @UnknownKeyFor @NonNull @Initialized OutputStream outStream) - throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { - - } - - @Override - public RangeMap decode( - @UnknownKeyFor @NonNull @Initialized InputStream inStream) - throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { - return TreeRangeMap.create(); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized List> getCoderArguments() { - return new ArrayList>>(); - } - - @Override - public void verifyDeterministic() - throws @UnknownKeyFor @NonNull @Initialized NonDeterministicException { - - } - } - - @Override - public @UnknownKeyFor @NonNull @Initialized Coder> getAccumulatorCoder( - @UnknownKeyFor @NonNull @Initialized CoderRegistry registry, - @UnknownKeyFor @NonNull @Initialized Coder inputCoder) - throws @UnknownKeyFor @NonNull @Initialized CannotProvideCoderException { - return new AccumulatorCoder(); - } - } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java index 9e8205789edb..857ce0503f24 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java @@ -2,7 +2,6 @@ import org.apache.beam.sdk.coders.BooleanCoder; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.extensions.ordered.GlobalSequenceTracker.SequenceAndTimestamp; import org.apache.beam.sdk.extensions.ordered.ProcessingState.ProcessingStateCoder; import org.apache.beam.sdk.state.OrderedListState; import org.apache.beam.sdk.state.StateSpec; @@ -54,7 +53,7 @@ class GlobalSequencesProcessorDoFn latestContinuousSequenceSideInput; + private final PCollectionView latestContinuousSequenceSideInput; /** * Stateful DoFn to do the bulk of processing. @@ -81,7 +80,7 @@ class GlobalSequencesProcessorDoFn>>> unprocessedEventTupleTag, boolean produceStatusUpdateOnEveryEvent, long maxNumberOfResultsToProduce, - PCollectionView latestContinuousSequenceSideInput) { + PCollectionView latestContinuousSequenceSideInput) { super(eventExaminer, mainOutputTupleTag, statusTupleTag, statusUpdateFrequency, unprocessedEventTupleTag, produceStatusUpdateOnEveryEvent, maxNumberOfResultsToProduce); @@ -115,7 +114,7 @@ public void processElement(ProcessContext context, MultiOutputReceiver outputReceiver, BoundedWindow window) { - SequenceAndTimestamp lastContinuousSequence = context.sideInput( + CompletedSequenceRange lastContinuousSequence = context.sideInput( latestContinuousSequenceSideInput); EventT event = eventAndSequence.getValue().getValue(); @@ -180,7 +179,7 @@ public void processElement(ProcessContext context, private void setBatchEmissionTimerIfNeeded(Timer batchEmissionTimer, ProcessingState processingState) { - SequenceAndTimestamp lastCompleteGlobalSequence = processingState.getLastCompleteGlobalSequence(); + CompletedSequenceRange lastCompleteGlobalSequence = processingState.getLastCompleteGlobalSequence(); if (processingState.getBufferedEventCount() > 0 && lastCompleteGlobalSequence != null) { batchEmissionTimer.set(lastCompleteGlobalSequence.getTimestamp()); } @@ -205,7 +204,7 @@ public void onBatchEmission( StateT state = mutableStateState.read(); - SequenceAndTimestamp lastCompleteGlobalSequence = processingState.getLastCompleteGlobalSequence(); + CompletedSequenceRange lastCompleteGlobalSequence = processingState.getLastCompleteGlobalSequence(); if (lastCompleteGlobalSequence == null) { LOG.warn("Last complete global instance is null."); return; @@ -219,8 +218,10 @@ public void onBatchEmission( LOG.info("Emission timer: " + processingState); + this.numberOfResultsBeforeBundleStart = processingState.getResultCount(); + state = processBufferedEventRange(processingState, state, bufferedEventsState, outputReceiver, - batchEmissionTimer, lastCompleteGlobalSequence.getSequence()); + batchEmissionTimer, lastCompleteGlobalSequence); saveStates( processingStatusState, diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java index 2b49ef9cee24..7abb92e75582 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java @@ -25,7 +25,6 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.VarLongCoder; -import org.apache.beam.sdk.extensions.ordered.GlobalSequenceTracker.SequenceAndTimestamp; import org.apache.beam.sdk.extensions.ordered.UnprocessedEvent.UnprocessedEventCoder; import org.apache.beam.sdk.schemas.NoSuchSchemaException; import org.apache.beam.sdk.schemas.SchemaCoder; @@ -39,6 +38,7 @@ import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PCollectionTuple; 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.TupleTagList; import org.apache.beam.sdk.values.TypeDescriptor; @@ -126,11 +126,12 @@ public OrderedEventProcessorResult expand( switch (handler.getSequenceType()) { case GLOBAL: - final PCollectionView latestContinuousSequence = + final PCollectionView latestContinuousSequence = input .apply("Convert to SequenceAndTimestamp", - ParDo.of(new EventToSequenceAndTimestampConverter<>())) - .apply("Global Sequence Tracker", new GlobalSequenceTracker()); + ParDo.of(new ToTimestampedEventConverter<>())) + .apply("Global Sequence Tracker", + new GlobalSequenceTracker<>(handler.getEventExaminer())); PCollection>> tickers = input.apply("Create Tickers", new PerKeyTickerGenerator<>(keyCoder, eventCoder, Duration.standardSeconds(5))); @@ -219,14 +220,14 @@ private static Coder getOrderedProcessingStatusCoder(Pi return result; } - static class EventToSequenceAndTimestampConverter - extends DoFn>, SequenceAndTimestamp> { + static class ToTimestampedEventConverter extends + DoFn>, TimestampedValue>>> { @ProcessElement public void convert(@Element KV> element, - @Timestamp Instant timestamp, OutputReceiver outputReceiver) { - outputReceiver.output( - SequenceAndTimestamp.create(element.getValue().getKey(), timestamp)); + @Timestamp Instant timestamp, + OutputReceiver>>> outputReceiver) { + outputReceiver.output(TimestampedValue.of(element, timestamp)); } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java index 8f8309d340b6..997e2910278e 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java @@ -54,6 +54,7 @@ public abstract class OrderedProcessingHandler< private boolean produceStatusUpdateOnEveryEvent = DEFAULT_PRODUCE_STATUS_UPDATE_ON_EVERY_EVENT; private SequenceType sequenceType = SequenceType.PER_KEY; + private @Nullable Long initialGlobalSequence; /** * Provide concrete classes which will be used by the ordered processing transform. @@ -72,6 +73,7 @@ public OrderedProcessingHandler( this.keyTClass = keyTClass; this.stateTClass = stateTClass; this.resultTClass = resultTClass; + this.initialGlobalSequence = null; } /** @return the event examiner instance which will be used by the transform. */ @@ -227,4 +229,12 @@ public SequenceType getSequenceType() { public void setSequenceType(SequenceType sequenceType) { this.sequenceType = sequenceType; } + + public @javax.annotation.Nullable Long getInitialGlobalSequence() { + return initialGlobalSequence; + } + + public void setInitialGlobalSequence(Long initialGlobalSequence) { + this.initialGlobalSequence = initialGlobalSequence; + } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java index 6c0f3df9d346..b04b984b0070 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java @@ -28,9 +28,7 @@ import org.apache.beam.sdk.coders.NullableCoder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; -import org.apache.beam.sdk.extensions.ordered.GlobalSequenceTracker.SequenceAndTimestamp; import org.apache.beam.sdk.extensions.ordered.GlobalSequenceTracker.SequenceAndTimestampCoder; -import org.apache.beam.sdk.schemas.SchemaCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.initialization.qual.Initialized; @@ -54,7 +52,7 @@ class ProcessingState { private long resultCount; - @Nullable private SequenceAndTimestamp lastCompleteGlobalSequence; + @Nullable private CompletedSequenceRange lastCompleteGlobalSequence; private KeyT key; @@ -136,11 +134,11 @@ public KeyT getKey() { return key; } - public @Nullable SequenceAndTimestamp getLastCompleteGlobalSequence() { + public @Nullable CompletedSequenceRange getLastCompleteGlobalSequence() { return lastCompleteGlobalSequence; } - public void setLastCompleteGlobalSequence(@Nullable SequenceAndTimestamp lastCompleteGlobalSequence) { + public void setLastCompleteGlobalSequence(@Nullable CompletedSequenceRange lastCompleteGlobalSequence) { this.lastCompleteGlobalSequence = lastCompleteGlobalSequence; } @@ -304,7 +302,7 @@ public long resultsProducedInBundle(long numberOfResultsBeforeBundleStart) { return resultCount - numberOfResultsBeforeBundleStart; } - public void updateGlobalSequenceDetails(SequenceAndTimestamp updated) { + public void updateGlobalSequenceDetails(CompletedSequenceRange updated) { // TODO: do we need to select max? Do we care about the timestamp? this.lastCompleteGlobalSequence = updated; } @@ -322,7 +320,7 @@ static class ProcessingStateCoder extends Coder> { private static final VarIntCoder INTEGER_CODER = VarIntCoder.of(); private static final BooleanCoder BOOLEAN_CODER = BooleanCoder.of(); - private static final NullableCoder SEQUENCE_AND_TIMESTAMP_CODER = + private static final NullableCoder SEQUENCE_AND_TIMESTAMP_CODER = NullableCoder.of(SequenceAndTimestampCoder.of()); private Coder keyCoder; @@ -360,7 +358,7 @@ public ProcessingState decode(InputStream inStream) throws IOException { long resultCount = LONG_CODER.decode(inStream); boolean isLastEventReceived = BOOLEAN_CODER.decode(inStream); KeyT key = keyCoder.decode(inStream); - SequenceAndTimestamp lastCompleteGlobalSequence = SEQUENCE_AND_TIMESTAMP_CODER.decode(inStream); + CompletedSequenceRange lastCompleteGlobalSequence = SEQUENCE_AND_TIMESTAMP_CODER.decode(inStream); ProcessingState result = new ProcessingState<>( key, diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java index 76a2097ec006..16381c470183 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java @@ -294,7 +294,7 @@ private void bufferEvent(long currentSequence, EventTypeT currentEvent, @Nullable StateTypeT processBufferedEventRange(ProcessingState processingState, @Nullable StateTypeT state, OrderedListState bufferedEventsState, MultiOutputReceiver outputReceiver, - Timer largeBatchEmissionTimer, long lastKnownCompleteSequenceNumber) { + Timer largeBatchEmissionTimer, CompletedSequenceRange completedSequenceRange) { Long earliestBufferedSequence = processingState.getEarliestBufferedSequence(); Long latestBufferedSequence = processingState.getLatestBufferedSequence(); if(earliestBufferedSequence == null || latestBufferedSequence == null) { @@ -332,7 +332,8 @@ private void bufferEvent(long currentSequence, EventTypeT currentEvent, boolean currentEventIsNotInSequence = lastOutputSequence != null && eventSequence > lastOutputSequence + 1; boolean stopProcessing = checkForSequenceGapInBufferedEvents() ? currentEventIsNotInSequence : - (! (eventSequence <= lastKnownCompleteSequenceNumber) && currentEventIsNotInSequence); + // TODO: can it be made more clear? + (! (eventSequence <= completedSequenceRange.getEnd()) && currentEventIsNotInSequence); if (stopProcessing) { processingState.foundSequenceGap(eventSequence); // Records will be cleared up to this element diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java index 077fade4417e..534cccae0001 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java @@ -193,7 +193,7 @@ private void processBufferedEvents( } processBufferedEventRange(processingState, state, bufferedEventsState, outputReceiver, - largeBatchEmissionTimer, Long.MIN_VALUE); + largeBatchEmissionTimer, CompletedSequenceRange.EMPTY); } @OnTimer(LARGE_BATCH_EMISSION_TIMER) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/UnprocessedEvent.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/UnprocessedEvent.java index 2131ef384e22..3c8f655ffca6 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/UnprocessedEvent.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/UnprocessedEvent.java @@ -68,6 +68,7 @@ static UnprocessedEvent create( return new AutoValue_UnprocessedEvent<>(event, reason, failureDetails); } + // TODO: do we need another reason, "before the initial event"? public enum Reason { duplicate, buffered, diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java new file mode 100644 index 000000000000..f2b936589a40 --- /dev/null +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java @@ -0,0 +1,77 @@ +package org.apache.beam.sdk.extensions.ordered.combiner; + +import java.util.Iterator; +import java.util.function.BiFunction; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.extensions.ordered.CompletedSequenceRange; +import org.apache.beam.sdk.extensions.ordered.EventExaminer; +import org.apache.beam.sdk.extensions.ordered.MutableState; +import org.apache.beam.sdk.extensions.ordered.combiner.SequenceRangeAccumulator.SequenceRangeAccumulatorCoder; +import org.apache.beam.sdk.transforms.Combine.CombineFn; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TimestampedValue; +import org.checkerframework.checker.initialization.qual.Initialized; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.checkerframework.checker.nullness.qual.UnknownKeyFor; +import org.joda.time.Instant; + +public class DefaultSequenceCombiner> extends + CombineFn>>, SequenceRangeAccumulator, CompletedSequenceRange> { + + public static final BiFunction<@NonNull Instant, @Nullable Instant, @Nullable Instant> OLDEST_TIMESTAMP_SELECTOR = (instant1, instant2) -> { + if (instant2 == null) { + return instant1; + } + @NonNull Instant nonNullableSecondValue = instant2; + return instant1.isAfter(nonNullableSecondValue) ? instant1 : nonNullableSecondValue; + }; + private final EventExaminer eventExaminer; + + public DefaultSequenceCombiner(EventExaminer eventExaminer) { + this.eventExaminer = eventExaminer; + } + + @Override + public SequenceRangeAccumulator createAccumulator() { + return new SequenceRangeAccumulator(); + } + + @Override + public SequenceRangeAccumulator addInput(SequenceRangeAccumulator accum, + TimestampedValue>> event) { + long sequence = event.getValue().getValue().getKey(); + + accum.add(sequence, event.getTimestamp(), + eventExaminer.isInitialEvent(sequence, event.getValue().getValue().getValue())); + + return accum; + } + + @Override + public SequenceRangeAccumulator mergeAccumulators( + Iterable accumulators) { + // There should be at least one accumulator. + Iterator iterator = accumulators.iterator(); + SequenceRangeAccumulator result = iterator.next(); + while (iterator.hasNext()) { + result.merge(iterator.next()); + } + return result; + } + + @Override + public CompletedSequenceRange extractOutput(SequenceRangeAccumulator accum) { + return accum.largestContinuousRange(); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized Coder getAccumulatorCoder( + @UnknownKeyFor @NonNull @Initialized CoderRegistry registry, + @UnknownKeyFor @NonNull @Initialized Coder>>> inputCoder) + throws @UnknownKeyFor @NonNull @Initialized CannotProvideCoderException { + return new SequenceRangeAccumulatorCoder(); + } +} diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java new file mode 100644 index 000000000000..25ddfda63a19 --- /dev/null +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java @@ -0,0 +1,78 @@ +package org.apache.beam.sdk.extensions.ordered.combiner; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Map.Entry; +import java.util.Objects; +import java.util.TreeMap; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.extensions.ordered.CompletedSequenceRange; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.commons.lang3.tuple.Triple; +import org.checkerframework.checker.initialization.qual.Initialized; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.UnknownKeyFor; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SequenceRangeAccumulator { + + private static final Logger LOG = LoggerFactory.getLogger(SequenceRangeAccumulator.class); + + private final TreeMap> accumulator = new TreeMap<>(); + private boolean containsInitialSequence = false; + + public void add(long sequence, Instant timestamp, boolean initialSequence) { + if (containsInitialSequence && initialSequence) { + // TODO: more tests + LOG.error( + "There are multiple initial sequences detected: " + + accumulator.keySet().iterator().next() + + " and " + initialSequence); + } + + if (initialSequence) { + this.containsInitialSequence = initialSequence; + } + } + + public CompletedSequenceRange largestContinuousRange() { + if (!containsInitialSequence) { + return CompletedSequenceRange.EMPTY; + } + + Entry> firstEntry = accumulator.firstEntry(); + if(firstEntry == null) { + throw new IllegalStateException("First entry is null"); + } + Long key = firstEntry.getKey(); + + return CompletedSequenceRange.create( + key, firstEntry.getValue().getLeft(), firstEntry.getValue().getRight()); + } + + public void merge(SequenceRangeAccumulator another) { + } + + public static class SequenceRangeAccumulatorCoder extends CustomCoder { + // TODO implement + @Override + public void encode(SequenceRangeAccumulator value, + @UnknownKeyFor @NonNull @Initialized OutputStream outStream) + throws @UnknownKeyFor@NonNull@Initialized CoderException, @UnknownKeyFor@NonNull@Initialized IOException { + + } + + @Override + public SequenceRangeAccumulator decode( + @UnknownKeyFor @NonNull @Initialized InputStream inStream) + throws @UnknownKeyFor@NonNull@Initialized CoderException, @UnknownKeyFor@NonNull@Initialized IOException { + return new SequenceRangeAccumulator(); + } + } +} diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/StringBufferOrderedProcessingHandler.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/StringBufferOrderedProcessingHandler.java index 72f3a3cf21b6..d70001369c16 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/StringBufferOrderedProcessingHandler.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/StringBufferOrderedProcessingHandler.java @@ -32,6 +32,8 @@ public class StringBufferOrderedProcessingHandler public StringBufferOrderedProcessingHandler(int emissionFrequency, long initialSequence) { super(String.class, String.class, StringBuilderState.class, String.class); this.eventExaminer = new StringEventExaminer(initialSequence, emissionFrequency); + // needed when using global sequences + setInitialGlobalSequence(initialSequence); } @Override diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java new file mode 100644 index 000000000000..046384ea702b --- /dev/null +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java @@ -0,0 +1,46 @@ +package org.apache.beam.sdk.extensions.ordered.combiner; + +import java.util.Arrays; +import org.apache.commons.lang3.tuple.Triple; +import org.joda.time.Instant; +import org.junit.Assert; +import org.junit.Test; + +public class SequenceRangeAccumulatorTest { + + static class Event { + + long sequence; + Instant timestamp; + boolean initialEvent; + + Event(long sequence, Instant ts) { + this(sequence, ts, false); + } + + Event(long sequence, Instant ts, boolean initialEvent) { + this.sequence = sequence; + this.timestamp = ts; + this.initialEvent = initialEvent; + } + } + + @Test + public void testSimpleAccumulation() { + Instant start = Instant.now(); + Event[] events = new Event[]{ + new Event(1, start, true), + new Event(2, start), + new Event(3, start) + }; + + SequenceRangeAccumulator accumulator = new SequenceRangeAccumulator(); + Arrays.stream(events).forEach(e -> accumulator.add(e.sequence, e.timestamp, e.initialEvent)); + + Assert.assertEquals("Accumulated results", + Triple.of(1, 3, start), + accumulator.largestContinuousRange()); + + } + +} From ed1b018ed9cdea064151f16b91d9cc567c141646 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Tue, 3 Sep 2024 08:00:04 -0700 Subject: [PATCH 08/33] First iteration of the efficient merging accumulator --- sdks/java/extensions/ordered/build.gradle | 1 + .../ordered/CompletedSequenceRange.java | 4 +- .../ordered/GlobalSequenceTracker.java | 2 +- .../combiner/SequenceRangeAccumulator.java | 54 +++++++++++++++---- .../SequenceRangeAccumulatorTest.java | 40 +++++++++++++- 5 files changed, 87 insertions(+), 14 deletions(-) diff --git a/sdks/java/extensions/ordered/build.gradle b/sdks/java/extensions/ordered/build.gradle index 338400c03554..2338b4f72b47 100644 --- a/sdks/java/extensions/ordered/build.gradle +++ b/sdks/java/extensions/ordered/build.gradle @@ -30,5 +30,6 @@ dependencies { testImplementation library.java.hamcrest testImplementation library.java.slf4j_jdk14 testImplementation project(path: ':sdks:java:core') + testImplementation 'junit:junit:4.13.1' testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") } \ No newline at end of file diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/CompletedSequenceRange.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/CompletedSequenceRange.java index 230e5b920fbd..78b50489b8d0 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/CompletedSequenceRange.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/CompletedSequenceRange.java @@ -9,7 +9,7 @@ @AutoValue public abstract class CompletedSequenceRange { public static final CompletedSequenceRange EMPTY = - CompletedSequenceRange.create(Long.MIN_VALUE, Long.MIN_VALUE, Instant.ofEpochMilli( + CompletedSequenceRange.of(Long.MIN_VALUE, Long.MIN_VALUE, Instant.ofEpochMilli( Long.MIN_VALUE)); public abstract long getStart(); @@ -20,7 +20,7 @@ public boolean isEmpty() { return this.equals(EMPTY); } - public static CompletedSequenceRange create(long start, long end, Instant timestamp) { + public static CompletedSequenceRange of(long start, long end, Instant timestamp) { return new AutoValue_CompletedSequenceRange(start, end, timestamp); } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java index 2601203cf794..050d5fd36641 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java @@ -82,7 +82,7 @@ public CompletedSequenceRange decode(@UnknownKeyFor @NonNull @Initialized InputS long start = VarLongCoder.of().decode(inStream); long end = VarLongCoder.of().decode(inStream); Instant timestamp = InstantCoder.of().decode(inStream); - return CompletedSequenceRange.create(start, end, timestamp); + return CompletedSequenceRange.of(start, end, timestamp); } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java index 25ddfda63a19..377eeba871b1 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java @@ -4,15 +4,11 @@ import java.io.InputStream; import java.io.OutputStream; import java.util.Map.Entry; -import java.util.Objects; import java.util.TreeMap; -import javax.annotation.Nullable; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.extensions.ordered.CompletedSequenceRange; import org.apache.commons.lang3.tuple.Pair; -import org.apache.commons.lang3.tuple.Triple; import org.checkerframework.checker.initialization.qual.Initialized; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.UnknownKeyFor; @@ -39,6 +35,38 @@ public void add(long sequence, Instant timestamp, boolean initialSequence) { if (initialSequence) { this.containsInitialSequence = initialSequence; } + + Entry> entry = accumulator.floorEntry(sequence); + if (entry == null) { + // Value lower than anything we have seen - this is a new range + accumulator.put(sequence, Pair.of(sequence, timestamp)); + return; + } + + Long inclusiveUpperBoundary = entry.getValue().getLeft(); + + if (sequence <= inclusiveUpperBoundary) { + // Duplicate + return; + } + + if (inclusiveUpperBoundary + 1 == sequence) { + // We hit the end of the range + + Pair rangeToMergeWith = accumulator.get(inclusiveUpperBoundary + 2); + if(rangeToMergeWith == null) { + accumulator.replace(entry.getKey(), Pair.of(sequence, timestamp)); + return; + } else { + accumulator.remove(inclusiveUpperBoundary + 2); + accumulator.replace(entry.getKey(), Pair.of(rangeToMergeWith.getKey(), timestamp)); + } + + return; + } + + // we are above the range. It's a new range + accumulator.put(sequence, Pair.of(sequence, timestamp)); } public CompletedSequenceRange largestContinuousRange() { @@ -47,31 +75,37 @@ public CompletedSequenceRange largestContinuousRange() { } Entry> firstEntry = accumulator.firstEntry(); - if(firstEntry == null) { + if (firstEntry == null) { throw new IllegalStateException("First entry is null"); } - Long key = firstEntry.getKey(); + Long startingSequence = firstEntry.getKey(); - return CompletedSequenceRange.create( - key, firstEntry.getValue().getLeft(), firstEntry.getValue().getRight()); + return CompletedSequenceRange.of( + startingSequence, firstEntry.getValue().getLeft(), firstEntry.getValue().getRight()); } + public int numberOfRanges() { + return accumulator.size(); + } + + public void merge(SequenceRangeAccumulator another) { } public static class SequenceRangeAccumulatorCoder extends CustomCoder { + // TODO implement @Override public void encode(SequenceRangeAccumulator value, @UnknownKeyFor @NonNull @Initialized OutputStream outStream) - throws @UnknownKeyFor@NonNull@Initialized CoderException, @UnknownKeyFor@NonNull@Initialized IOException { + throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { } @Override public SequenceRangeAccumulator decode( @UnknownKeyFor @NonNull @Initialized InputStream inStream) - throws @UnknownKeyFor@NonNull@Initialized CoderException, @UnknownKeyFor@NonNull@Initialized IOException { + throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { return new SequenceRangeAccumulator(); } } diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java index 046384ea702b..204187c24e29 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java @@ -1,6 +1,7 @@ package org.apache.beam.sdk.extensions.ordered.combiner; import java.util.Arrays; +import org.apache.beam.sdk.extensions.ordered.CompletedSequenceRange; import org.apache.commons.lang3.tuple.Triple; import org.joda.time.Instant; import org.junit.Assert; @@ -34,13 +35,50 @@ public void testSimpleAccumulation() { new Event(3, start) }; + doTest(events, CompletedSequenceRange.of(1, 3, start), 1); + } + + @Test + public void testPartialRangeAccumulation() { + Instant start = Instant.now(); + Event[] events = new Event[]{ + new Event(1, start, true), + new Event(2, start), + new Event(3, start), + new Event(5, start), + new Event(7, start), + + }; + + doTest(events, CompletedSequenceRange.of(1, 3, start), 3); + } + + @Test + public void testMergingRangeAccumulation() { + Instant start = Instant.now(); + Event[] events = new Event[]{ + new Event(1, start, true), + new Event(2, start), + new Event(3, start), + new Event(5, start), + new Event(7, start), + new Event(6, start), + }; + + doTest(events, CompletedSequenceRange.of(1, 3, start), 2); + } + + private static void doTest(Event[] events, CompletedSequenceRange expectedResult, + int expectedNumberOfRanges) { SequenceRangeAccumulator accumulator = new SequenceRangeAccumulator(); Arrays.stream(events).forEach(e -> accumulator.add(e.sequence, e.timestamp, e.initialEvent)); Assert.assertEquals("Accumulated results", - Triple.of(1, 3, start), + expectedResult, accumulator.largestContinuousRange()); + Assert.assertEquals("Number of ranges", expectedNumberOfRanges, accumulator.numberOfRanges()); } + } From 3094ec943f4425cacc436a22135c24a0ce354898 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Sun, 8 Sep 2024 21:57:16 -0700 Subject: [PATCH 09/33] Mostly complete implementation of the accumulator and corresponding tests. --- .../ordered/CompletedSequenceRange.java | 43 ++- .../ordered/GlobalSequenceTracker.java | 55 +--- .../extensions/ordered/ProcessingState.java | 3 +- .../combiner/SequenceRangeAccumulator.java | 169 ++++++++--- .../SequenceRangeAccumulatorTest.java | 269 ++++++++++++++++-- 5 files changed, 423 insertions(+), 116 deletions(-) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/CompletedSequenceRange.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/CompletedSequenceRange.java index 78b50489b8d0..cacfd029f9e7 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/CompletedSequenceRange.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/CompletedSequenceRange.java @@ -1,11 +1,22 @@ package org.apache.beam.sdk.extensions.ordered; import com.google.auto.value.AutoValue; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.coders.InstantCoder; +import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.extensions.ordered.CompletedSequenceRange.CompletedSequenceRangeCoder; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.checkerframework.checker.initialization.qual.Initialized; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.UnknownKeyFor; import org.joda.time.Instant; -@DefaultSchema(AutoValueSchema.class) @AutoValue public abstract class CompletedSequenceRange { public static final CompletedSequenceRange EMPTY = @@ -23,4 +34,34 @@ public boolean isEmpty() { public static CompletedSequenceRange of(long start, long end, Instant timestamp) { return new AutoValue_CompletedSequenceRange(start, end, timestamp); } + + static class CompletedSequenceRangeCoder extends CustomCoder { + + private static final CompletedSequenceRangeCoder INSTANCE = new CompletedSequenceRangeCoder(); + + static CompletedSequenceRangeCoder of() { + return INSTANCE; + } + + private CompletedSequenceRangeCoder() { + } + + @Override + public void encode(CompletedSequenceRange value, + @UnknownKeyFor @NonNull @Initialized OutputStream outStream) + throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { + VarLongCoder.of().encode(value.getStart(), outStream); + VarLongCoder.of().encode(value.getEnd(), outStream); + InstantCoder.of().encode(value.getTimestamp(), outStream); + } + + @Override + public CompletedSequenceRange decode(@UnknownKeyFor @NonNull @Initialized InputStream inStream) + throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { + long start = VarLongCoder.of().decode(inStream); + long end = VarLongCoder.of().decode(inStream); + Instant timestamp = InstantCoder.of().decode(inStream); + return CompletedSequenceRange.of(start, end, timestamp); + } + } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java index 050d5fd36641..55437749e759 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java @@ -1,14 +1,7 @@ package org.apache.beam.sdk.extensions.ordered; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CustomCoder; -import org.apache.beam.sdk.coders.InstantCoder; -import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.extensions.ordered.CompletedSequenceRange.CompletedSequenceRangeCoder; import org.apache.beam.sdk.extensions.ordered.combiner.DefaultSequenceCombiner; -import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.AfterFirst; @@ -17,17 +10,11 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Repeatedly; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.Preconditions; 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.TimestampedValue; -import org.checkerframework.checker.initialization.qual.Initialized; -import org.checkerframework.checker.nullness.qual.NonNull; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.checkerframework.checker.nullness.qual.UnknownKeyFor; import org.joda.time.Duration; -import org.joda.time.Instant; class GlobalSequenceTracker> extends PTransform>>>, PCollectionView> { @@ -41,49 +28,21 @@ public GlobalSequenceTracker(EventExaminer eventExaminer) { @Override public PCollectionView expand( PCollection>>> input) { + input.getPipeline().getCoderRegistry().registerCoderForClass( + CompletedSequenceRange.class, + CompletedSequenceRangeCoder.of()); return input // TODO: get the windowing strategy from the input rather than assume global windows. .apply("Setup Triggering", - Window.>>>into(new GlobalWindows()) + Window.>>>into( + new GlobalWindows()) .accumulatingFiredPanes() .triggering( Repeatedly.forever(AfterFirst.of( AfterPane.elementCountAtLeast(1), AfterProcessingTime.pastFirstElementInPane() .plusDelayOf(Duration.standardSeconds(5)))))) - .apply("Create Side Input", - Combine.globally(sequenceCombiner).asSingletonView()); + .apply("Create Side Input", Combine.globally(sequenceCombiner).asSingletonView()); } - - static class SequenceAndTimestampCoder extends CustomCoder { - - private static final SequenceAndTimestampCoder INSTANCE = new SequenceAndTimestampCoder(); - - static SequenceAndTimestampCoder of() { - return INSTANCE; - } - - private SequenceAndTimestampCoder() { - } - - @Override - public void encode(CompletedSequenceRange value, - @UnknownKeyFor @NonNull @Initialized OutputStream outStream) - throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { - VarLongCoder.of().encode(value.getStart(), outStream); - VarLongCoder.of().encode(value.getEnd(), outStream); - InstantCoder.of().encode(value.getTimestamp(), outStream); - } - - @Override - public CompletedSequenceRange decode(@UnknownKeyFor @NonNull @Initialized InputStream inStream) - throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { - long start = VarLongCoder.of().decode(inStream); - long end = VarLongCoder.of().decode(inStream); - Instant timestamp = InstantCoder.of().decode(inStream); - return CompletedSequenceRange.of(start, end, timestamp); - } - } - } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java index b04b984b0070..04efc40df9f8 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java @@ -28,7 +28,6 @@ import org.apache.beam.sdk.coders.NullableCoder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; -import org.apache.beam.sdk.extensions.ordered.GlobalSequenceTracker.SequenceAndTimestampCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.initialization.qual.Initialized; @@ -321,7 +320,7 @@ static class ProcessingStateCoder extends Coder> { private static final BooleanCoder BOOLEAN_CODER = BooleanCoder.of(); private static final NullableCoder SEQUENCE_AND_TIMESTAMP_CODER = - NullableCoder.of(SequenceAndTimestampCoder.of()); + NullableCoder.of(CompletedSequenceRange.CompletedSequenceRangeCoder.of()); private Coder keyCoder; diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java index 377eeba871b1..60d42a90387d 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java @@ -4,7 +4,10 @@ import java.io.InputStream; import java.io.OutputStream; import java.util.Map.Entry; +import java.util.Objects; +import java.util.SortedMap; import java.util.TreeMap; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.extensions.ordered.CompletedSequenceRange; @@ -13,64 +16,78 @@ import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.UnknownKeyFor; import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class SequenceRangeAccumulator { - private static final Logger LOG = LoggerFactory.getLogger(SequenceRangeAccumulator.class); + static Instant max(Instant a, Instant b) { + return a.isAfter(b) ? a : b; + } private final TreeMap> accumulator = new TreeMap<>(); - private boolean containsInitialSequence = false; - - public void add(long sequence, Instant timestamp, boolean initialSequence) { - if (containsInitialSequence && initialSequence) { - // TODO: more tests - LOG.error( - "There are multiple initial sequences detected: " - + accumulator.keySet().iterator().next() - + " and " + initialSequence); - } - - if (initialSequence) { - this.containsInitialSequence = initialSequence; - } + private @Nullable Long initialSequence = null; - Entry> entry = accumulator.floorEntry(sequence); - if (entry == null) { - // Value lower than anything we have seen - this is a new range - accumulator.put(sequence, Pair.of(sequence, timestamp)); - return; + public void add(long sequence, Instant timestamp, boolean isInitialSequence) { + if (isInitialSequence && this.initialSequence != null && sequence != this.initialSequence) { + throw new IllegalStateException( + "There are different initial sequences detected: " + + initialSequence + " and " + sequence); } - Long inclusiveUpperBoundary = entry.getValue().getLeft(); - - if (sequence <= inclusiveUpperBoundary) { - // Duplicate + if (isInitialSequence) { + this.initialSequence = sequence; + clearRangesBelowInitialSequence(sequence, timestamp); + } else if (initialSequence != null && sequence <= initialSequence) { + // No need to add anything lower than the initial sequence to the accumulator. return; } - if (inclusiveUpperBoundary + 1 == sequence) { - // We hit the end of the range + long lowerBound = sequence; + long upperBound = sequence; - Pair rangeToMergeWith = accumulator.get(inclusiveUpperBoundary + 2); - if(rangeToMergeWith == null) { - accumulator.replace(entry.getKey(), Pair.of(sequence, timestamp)); + Entry> lowerRange = accumulator.floorEntry(sequence); + if(lowerRange != null) { + long inclusiveUpperBoundary = lowerRange.getValue().getLeft(); + if (sequence <= inclusiveUpperBoundary) { + // Duplicate. No need to adjust the timestamp. return; - } else { - accumulator.remove(inclusiveUpperBoundary + 2); - accumulator.replace(entry.getKey(), Pair.of(rangeToMergeWith.getKey(), timestamp)); } - return; + if (inclusiveUpperBoundary + 1 == sequence) { + // The new element extends this range + timestamp = max(timestamp, lowerRange.getValue().getValue()); + lowerBound = lowerRange.getKey(); + accumulator.remove(lowerRange.getKey()); + } } - // we are above the range. It's a new range - accumulator.put(sequence, Pair.of(sequence, timestamp)); + long nextSequenceNumber = sequence + 1; + Pair upperRange = accumulator.get(nextSequenceNumber); + if (upperRange != null) { + timestamp = max(timestamp, upperRange.getRight()); + upperBound = upperRange.getLeft(); + accumulator.remove(nextSequenceNumber); + } + + accumulator.put(lowerBound, Pair.of(upperBound, timestamp)); + } + + private void clearRangesBelowInitialSequence(long sequence, Instant timestamp) { + // First, adjust the current range, if any + Entry> lowerRange = accumulator.floorEntry(sequence); + if(lowerRange != null + && lowerRange.getKey() < sequence + && lowerRange.getValue().getLeft() > sequence) { + // The sequence is in the middle of the range. Adjust it. + accumulator.remove(lowerRange.getKey()); + accumulator.put(sequence, + Pair.of(lowerRange.getValue().getKey(), max(timestamp, lowerRange.getValue() + .getValue()))); + } + accumulator.subMap(Long.MIN_VALUE, sequence).clear(); } public CompletedSequenceRange largestContinuousRange() { - if (!containsInitialSequence) { + if (initialSequence == null) { return CompletedSequenceRange.EMPTY; } @@ -78,10 +95,10 @@ public CompletedSequenceRange largestContinuousRange() { if (firstEntry == null) { throw new IllegalStateException("First entry is null"); } - Long startingSequence = firstEntry.getKey(); - - return CompletedSequenceRange.of( - startingSequence, firstEntry.getValue().getLeft(), firstEntry.getValue().getRight()); + Long start = firstEntry.getKey(); + Long end = firstEntry.getValue().getLeft(); + Instant latestTimestamp = firstEntry.getValue().getRight(); + return CompletedSequenceRange.of(start, end, latestTimestamp); } public int numberOfRanges() { @@ -90,6 +107,74 @@ public int numberOfRanges() { public void merge(SequenceRangeAccumulator another) { + if (this.initialSequence != null && another.initialSequence != null + && this.initialSequence.equals(another.initialSequence)) { + throw new IllegalStateException("Two accumulators contain different initial sequences: " + + this.initialSequence + " and " + another.initialSequence); + } + + if (another.initialSequence != null) { + long newInitialSequence = another.initialSequence; + this.initialSequence = newInitialSequence; + Entry> firstEntry = another.accumulator.firstEntry(); + if(firstEntry != null) { + Instant timestampOfTheInitialRange = firstEntry.getValue().getRight(); + clearRangesBelowInitialSequence(newInitialSequence, timestampOfTheInitialRange); + } + } + + another.accumulator.entrySet().stream().forEach( + entry -> { + long lowerBound = entry.getKey(); + long upperBound = entry.getValue().getLeft(); + if(this.initialSequence != null) { + if(upperBound < initialSequence) { + // The whole range is below the initial sequence. Ignore it. + return; + } + if(lowerBound < initialSequence) { + // This will cause pruning of the range up to the initial sequence + lowerBound = this.initialSequence; + } + } + + Entry> lowerRange = this.accumulator.floorEntry(lowerBound); + + if (lowerRange != null) { + if (lowerRange.getValue().getLeft() < lowerBound - 1) { + // Nothing to do. There is a lower non-adjacent range. + } else { + // We found an overlapping range and will replace it with a new one + upperBound = Math.max(upperBound, lowerRange.getValue().getLeft()); + lowerBound = lowerRange.getKey(); + } + } + + Entry> upperRange = this.accumulator.floorEntry(upperBound + 1); + if (upperRange == null || + (lowerRange != null && Objects.equals(upperRange.getKey(), lowerRange.getKey()))) { + // Nothing to do - either there is no adjacent upper range or it equals the lower range + } else { + upperBound = Math.max(upperBound, upperRange.getValue().getLeft()); + } + + Instant latestTimestamp = removeAllRanges(lowerBound, upperBound, + entry.getValue().getRight()); + + this.accumulator.put(lowerBound, Pair.of(upperBound, latestTimestamp)); + } + ); + } + + private Instant removeAllRanges(long lowerBound, long upperBound, Instant currentTimestamp) { + Instant result = currentTimestamp; + SortedMap> rangesToRemove = accumulator.subMap(lowerBound, + upperBound); + for (Pair value : rangesToRemove.values()) { + result = result.isAfter(value.getRight()) ? result : value.getRight(); + } + rangesToRemove.clear(); + return result; } public static class SequenceRangeAccumulatorCoder extends CustomCoder { diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java index 204187c24e29..c47f3ce8e18d 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java @@ -1,14 +1,30 @@ package org.apache.beam.sdk.extensions.ordered.combiner; import java.util.Arrays; +import java.util.concurrent.atomic.AtomicLong; import org.apache.beam.sdk.extensions.ordered.CompletedSequenceRange; -import org.apache.commons.lang3.tuple.Triple; import org.joda.time.Instant; import org.junit.Assert; import org.junit.Test; public class SequenceRangeAccumulatorTest { + // Atomic ust in case tests are run in parallel + private final static AtomicLong currentTicker = new AtomicLong(); + + static Instant nextTimestamp() { + return Instant.ofEpochMilli(currentTicker.getAndIncrement()); + } + + static Instant eventTimestamp(Event[] events, long eventSequence) { + for (Event e : events) { + if (e.sequence == eventSequence) { + return e.timestamp; + } + } + throw new IllegalStateException("Unable to find event with sequence " + eventSequence); + } + static class Event { long sequence; @@ -28,47 +44,117 @@ static class Event { @Test public void testSimpleAccumulation() { - Instant start = Instant.now(); Event[] events = new Event[]{ - new Event(1, start, true), - new Event(2, start), - new Event(3, start) + new Event(1, nextTimestamp(), true), + new Event(2, nextTimestamp()), + new Event(3, nextTimestamp()) }; - doTest(events, CompletedSequenceRange.of(1, 3, start), 1); + doTestAccumulation(events, CompletedSequenceRange.of(1, 3, eventTimestamp(events, 3)), 1); } @Test - public void testPartialRangeAccumulation() { - Instant start = Instant.now(); + public void testReverseArrivalHandling() { Event[] events = new Event[]{ - new Event(1, start, true), - new Event(2, start), - new Event(3, start), - new Event(5, start), - new Event(7, start), + new Event(3, nextTimestamp()), + new Event(2, nextTimestamp()), + new Event(1, nextTimestamp(), true) + }; + + Instant timestampOfEventNumber1 = eventTimestamp(events, 1); + doTestAccumulation(events, CompletedSequenceRange.of(1, 3, timestampOfEventNumber1), 1); + } + @Test + public void testPartialRangeAccumulation() { + Event[] events = new Event[]{ + new Event(1, nextTimestamp(), true), + new Event(2, nextTimestamp()), + new Event(3, nextTimestamp()), + new Event(5, nextTimestamp()), + new Event(7, nextTimestamp()), }; - doTest(events, CompletedSequenceRange.of(1, 3, start), 3); + doTestAccumulation(events, CompletedSequenceRange.of(1, 3, eventTimestamp(events, 3)), 3); } @Test public void testMergingRangeAccumulation() { - Instant start = Instant.now(); Event[] events = new Event[]{ - new Event(1, start, true), - new Event(2, start), - new Event(3, start), - new Event(5, start), - new Event(7, start), - new Event(6, start), + new Event(1, nextTimestamp(), true), + new Event(2, nextTimestamp()), + new Event(3, nextTimestamp()), + new Event(5, nextTimestamp()), + new Event(7, nextTimestamp()), + new Event(6, nextTimestamp()), + }; + + doTestAccumulation(events, CompletedSequenceRange.of(1, 3, eventTimestamp(events, 3)), 2); + } + + @Test + public void testNoStartEvent() { + Event[] events = new Event[]{ + new Event(2, nextTimestamp()), + new Event(3, nextTimestamp()), + new Event(1, nextTimestamp()), + + new Event(5, nextTimestamp()), }; - doTest(events, CompletedSequenceRange.of(1, 3, start), 2); + doTestAccumulation(events, CompletedSequenceRange.EMPTY, 2); + } + + @Test + public void testNoEventsAccumulation() { + Event[] events = new Event[]{}; + + doTestAccumulation(events, CompletedSequenceRange.EMPTY, 0); } - private static void doTest(Event[] events, CompletedSequenceRange expectedResult, + @Test + public void testRemovingRangesBelowInitialSequenceDuringAccumulation() { + Event[] events = new Event[]{ + // First range + new Event(2, nextTimestamp()), + new Event(3, nextTimestamp()), + new Event(1, nextTimestamp()), + + // Second range + new Event(5, nextTimestamp()), + new Event(6, nextTimestamp()), + + // This event should prune everything below + new Event(7, nextTimestamp(), true), + }; + + doTestAccumulation(events, CompletedSequenceRange.of(7,7, eventTimestamp(events, 7)), 1); + } + + @Test + public void testRemovingElementsBelowInitialSequenceDuringAccumulation() { + + Event[] events = new Event[]{ + // First range + new Event(2, nextTimestamp()), + new Event(3, nextTimestamp()), + new Event(1, nextTimestamp()), + + // Second range + new Event(5, nextTimestamp()), + new Event(6, nextTimestamp()), + new Event(7, nextTimestamp()), + new Event(8, nextTimestamp()), + + // This event should reduce the range. + new Event(7, nextTimestamp(), true), + }; + + Instant timestampOfTheLastEvent = events[events.length - 1].timestamp; + doTestAccumulation(events, CompletedSequenceRange.of(7,8, timestampOfTheLastEvent), 1); + } + + private static void doTestAccumulation(Event[] events, CompletedSequenceRange expectedResult, int expectedNumberOfRanges) { SequenceRangeAccumulator accumulator = new SequenceRangeAccumulator(); Arrays.stream(events).forEach(e -> accumulator.add(e.sequence, e.timestamp, e.initialEvent)); @@ -81,4 +167,141 @@ private static void doTest(Event[] events, CompletedSequenceRange expectedResult } + @Test + public void testEmptyMerge() { + Event[] set1 = new Event[]{}; + Event[] set2 = new Event[]{}; + + CompletedSequenceRange expectedResult = CompletedSequenceRange.EMPTY; + int expectedNumberOfRanges = 0; + + doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); + } + + @Test + public void testMergingNonEmptyWithEmpty() { + Event[] set1 = new Event[]{ + new Event(3, nextTimestamp()), + new Event(2, nextTimestamp()), + new Event(1, nextTimestamp(), true) + }; + Event[] set2 = new Event[]{}; + + CompletedSequenceRange expectedResult = CompletedSequenceRange.of(1, 3, + eventTimestamp(set1, 1L)); + int expectedNumberOfRanges = 1; + + doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); + } + + @Test + public void testMergingWithLowerNonAdjacentRange() { + Event[] set1 = new Event[]{ + new Event(1, nextTimestamp(), true), + new Event(2, nextTimestamp()), + }; + Event[] set2 = new Event[]{ + new Event(4, nextTimestamp()), + new Event(5, nextTimestamp()), + new Event(6, nextTimestamp()) + }; + + CompletedSequenceRange expectedResult = CompletedSequenceRange.of(1, 2, + eventTimestamp(set1, 2L)); + int expectedNumberOfRanges = 2; + + doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); + } + + @Test + public void testMergingWithoutAnyInitialEvents() { + Event[] set1 = new Event[]{ + new Event(1, nextTimestamp()), + new Event(2, nextTimestamp()), + }; + Event[] set2 = new Event[]{ + new Event(4, nextTimestamp()), + new Event(5, nextTimestamp()), + new Event(6, nextTimestamp()) + }; + + CompletedSequenceRange expectedResult = CompletedSequenceRange.EMPTY; + int expectedNumberOfRanges = 2; + + doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); + } + + @Test + public void testMergingAdjacentRanges() { + // TODO: Test the max timestamp being selected. + Event[] set1 = new Event[]{ + new Event(1, nextTimestamp(), true), + new Event(2, nextTimestamp()), + }; + Event[] set2 = new Event[]{ + new Event(3, nextTimestamp()), + new Event(4, nextTimestamp()), + new Event(5, nextTimestamp()), + new Event(6, nextTimestamp()) + }; + + CompletedSequenceRange expectedResult = CompletedSequenceRange.of(1, 6, + eventTimestamp(set2, 6L)); + int expectedNumberOfRanges = 1; + + doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); + } + + @Test + public void testPruningSequencesBelowInitial() { + Event[] set1 = new Event[]{ + new Event(1, nextTimestamp()), + new Event(2, nextTimestamp()), + }; + Event[] set2 = new Event[]{ + new Event(3, nextTimestamp(), true), + new Event(4, nextTimestamp()), + new Event(5, nextTimestamp()), + new Event(6, nextTimestamp()) + }; + + CompletedSequenceRange expectedResult = CompletedSequenceRange.of(3, 6, + eventTimestamp(set2, 6L)); + int expectedNumberOfRanges = 1; + + doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); + } + + @Test + public void testDuplicateHandling() { + // TODO: + } + + private static void doTestMerging(Event[] set1, Event[] set2, + CompletedSequenceRange expectedResult, + int expectedNumberOfRanges) { + // Try to merge both set2 to set1 and set1 to set2 - both must return the same results + mergeAndTest(set1, set2, expectedResult, expectedNumberOfRanges, "set1"); + mergeAndTest(set2, set1, expectedResult, expectedNumberOfRanges, "set2"); + } + + private static void mergeAndTest(Event[] set1, Event[] set2, + CompletedSequenceRange expectedResult, + int expectedNumberOfRanges, String firstSetName) { + final SequenceRangeAccumulator a1 = new SequenceRangeAccumulator(); + Arrays.stream(set1).forEach(e -> a1.add(e.sequence, e.timestamp, e.initialEvent)); + + final SequenceRangeAccumulator a2 = new SequenceRangeAccumulator(); + Arrays.stream(set2).forEach(e -> a2.add(e.sequence, e.timestamp, e.initialEvent)); + + a1.merge(a2); + + Assert.assertEquals("Accumulated results - " + firstSetName, + expectedResult, + a1.largestContinuousRange()); + + Assert.assertEquals("Number of ranges - " + firstSetName, expectedNumberOfRanges, + a1.numberOfRanges()); + } + } From 10a9deadaacb52641814bbf3c8000cebb5476bab Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Mon, 9 Sep 2024 15:25:45 -0700 Subject: [PATCH 10/33] Additional round of test refinements. --- .../extensions/ordered/ProcessingState.java | 1 - ...deredEventProcessorGlobalSequenceTest.java | 51 +++++++++++-------- ...deredEventProcessorPerKeySequenceTest.java | 2 +- ...ava => OrderedEventProcessorTestBase.java} | 2 +- .../StringBufferOrderedProcessingHandler.java | 2 +- 5 files changed, 33 insertions(+), 25 deletions(-) rename sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/{OrderedEventProcessorTest.java => OrderedEventProcessorTestBase.java} (99%) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java index 04efc40df9f8..120aca8cdaed 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java @@ -302,7 +302,6 @@ public long resultsProducedInBundle(long numberOfResultsBeforeBundleStart) { } public void updateGlobalSequenceDetails(CompletedSequenceRange updated) { - // TODO: do we need to select max? Do we care about the timestamp? this.lastCompleteGlobalSequence = updated; } diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java index 9135636f30db..12dcba6fd363 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java @@ -1,6 +1,7 @@ package org.apache.beam.sdk.extensions.ordered; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.List; import org.apache.beam.sdk.Pipeline; @@ -19,7 +20,7 @@ import org.joda.time.Instant; import org.junit.Test; -public class OrderedEventProcessorGlobalSequenceTest extends OrderedEventProcessorTest { +public class OrderedEventProcessorGlobalSequenceTest extends OrderedEventProcessorTestBase { @org.junit.Test public void testPerfectOrderingProcessing() throws CannotProvideCoderException { @@ -223,16 +224,19 @@ public void testSequenceGapProcessingInBufferedOutput() throws CannotProvideCode List events = new ArrayList<>(sequences.length); List> expectedOutput = new ArrayList<>(sequences.length); - StringBuilder output = new StringBuilder(); - String outputPerElement = "."; String key = "id-1"; for (long sequence : sequences) { - events.add(Event.create(sequence, key, outputPerElement)); - output.append(outputPerElement); - expectedOutput.add(KV.of(key, output.toString())); + events.add(Event.create(sequence, key, sequence + "-")); } + StringBuilder output = new StringBuilder(); + Arrays.stream(sequences).sorted().forEach(sequence -> { + output.append(sequence + "-"); + expectedOutput.add(KV.of(key, output.toString())); + } + ); + testGlobalSequenceProcessing( events.toArray(new Event[events.size()]), expectedOutput, @@ -273,6 +277,7 @@ public void testHandlingOfMaxSequenceNumber() throws CannotProvideCoderException @Test public void testProcessingOfTheLastInput() throws CannotProvideCoderException { + // TODO: fix the test. Need to see that the resulting status reflects the last input Event[] events = { Event.create(0, "id-1", "a"), Event.create(1, "id-1", "b"), @@ -374,6 +379,7 @@ public void testWindowedProcessing() throws CannotProvideCoderException { Event.create(0, "id-1", "c"), base.plus(Duration.standardSeconds(10))), TimestampedValue.of( Event.create(1, "id-1", "d"), base.plus(Duration.standardSeconds(11)))) + .advanceProcessingTime(Duration.standardMinutes(15)) .advanceWatermarkToInfinity(); Pipeline pipeline = streamingPipeline; @@ -389,7 +395,8 @@ public void testWindowedProcessing() throws CannotProvideCoderException { EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0); handler.setMaxOutputElementsPerBundle(LARGE_MAX_RESULTS_PER_OUTPUT); handler.setStatusUpdateFrequency(null); - handler.setProduceStatusUpdateOnEveryEvent(true); + handler.setProduceStatusUpdateOnEveryEvent(false); + handler.setSequenceType(SequenceType.GLOBAL); OrderedEventProcessor orderedEventProcessor = OrderedEventProcessor.create(handler); @@ -414,20 +421,22 @@ public void testWindowedProcessing() throws CannotProvideCoderException { .inWindow(window2) .containsInAnyOrder(KV.of("id-1", "c"), KV.of("id-1", "cd")); - PAssert.that("Statuses match in window 1", processingResult.processingStatuses()) - .inWindow(window1) - .containsInAnyOrder( - KV.of("id-1", OrderedProcessingStatus.create(0L, 0, null, null, 1, 1, 0, false)), - KV.of("id-1", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2, 0, false)), - KV.of("id-2", OrderedProcessingStatus.create(0L, 0, null, null, 1, 1, 0, false)), - KV.of("id-2", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2, 0, false)), - KV.of("id-2", OrderedProcessingStatus.create(2L, 0, null, null, 3, 3, 0, false))); - - PAssert.that("Statuses match in window 2", processingResult.processingStatuses()) - .inWindow(window2) - .containsInAnyOrder( - KV.of("id-1", OrderedProcessingStatus.create(0L, 0, null, null, 1, 1, 0, false)), - KV.of("id-1", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2, 0, false))); + // TODO: can we make the status assertions work? +// PAssert.that("Statuses match in window 1", processingResult.processingStatuses()) +// .inWindow(window1) +// .containsInAnyOrder( +//// KV.of("id-1", OrderedProcessingStatus.create(0L, 0, null, null, 1, 1, 0, false)), +// KV.of("id-1", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2, 0, false)), +//// KV.of("id-2", OrderedProcessingStatus.create(0L, 0, null, null, 1, 1, 0, false)), +//// KV.of("id-2", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2, 0, false)), +// KV.of("id-2", OrderedProcessingStatus.create(2L, 0, null, null, 3, 3, 0, false)) +// ); + +// PAssert.that("Statuses match in window 2", processingResult.processingStatuses()) +// .inWindow(window2) +// .containsInAnyOrder( +// KV.of("id-1", OrderedProcessingStatus.create(0L, 0, null, null, 1, 1, 0, false)), +// KV.of("id-1", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2, 0, false))); PAssert.that("Unprocessed events match", processingResult.unprocessedEvents()) .containsInAnyOrder(NO_EXPECTED_DLQ_EVENTS); diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java index 37ce671ba574..3cebf2019dac 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java @@ -20,7 +20,7 @@ import org.joda.time.Instant; import org.junit.Test; -public class OrderedEventProcessorPerKeySequenceTest extends OrderedEventProcessorTest { +public class OrderedEventProcessorPerKeySequenceTest extends OrderedEventProcessorTestBase { @Test public void testPerfectOrderingProcessing() throws CannotProvideCoderException { diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTestBase.java similarity index 99% rename from sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTest.java rename to sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTestBase.java index d140b8f4c6cd..2043b2cd0b59 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTestBase.java @@ -56,7 +56,7 @@ * Each event is a string for a particular key. The output is a concatenation of all strings. */ @RunWith(JUnit4.class) -public class OrderedEventProcessorTest { +public class OrderedEventProcessorTestBase { public static final boolean LAST_EVENT_RECEIVED = true; public static final int EMISSION_FREQUENCY_ON_EVERY_ELEMENT = 1; public static final int INITIAL_SEQUENCE_OF_0 = 0; diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/StringBufferOrderedProcessingHandler.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/StringBufferOrderedProcessingHandler.java index d70001369c16..95328981a10e 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/StringBufferOrderedProcessingHandler.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/StringBufferOrderedProcessingHandler.java @@ -32,7 +32,7 @@ public class StringBufferOrderedProcessingHandler public StringBufferOrderedProcessingHandler(int emissionFrequency, long initialSequence) { super(String.class, String.class, StringBuilderState.class, String.class); this.eventExaminer = new StringEventExaminer(initialSequence, emissionFrequency); - // needed when using global sequences + // needed when testing global sequences setInitialGlobalSequence(initialSequence); } From 03cfb3c5d996a17c29f8c1ade09f7a463692b346 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Tue, 10 Sep 2024 12:52:22 -0700 Subject: [PATCH 11/33] Added logic to DQL the records below the global sequence range. --- .../ordered/GlobalSequencesProcessorDoFn.java | 8 +++- .../sdk/extensions/ordered/ProcessorDoFn.java | 39 ++++++++++++++----- ...deredEventProcessorGlobalSequenceTest.java | 33 ++++++++++++++++ 3 files changed, 69 insertions(+), 11 deletions(-) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java index 857ce0503f24..657d37515c5a 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java @@ -122,7 +122,9 @@ public void processElement(ProcessContext context, EventKeyT key = eventAndSequence.getKey(); long sequence = eventAndSequence.getValue().getKey(); - LOG.info(key + ": " + sequence + " lastSequence: " + lastContinuousSequence); + if(LOG.isTraceEnabled()) { + LOG.trace(key + ": " + sequence + " lastSequence: " + lastContinuousSequence); + } ProcessingState processingState = processingStateState.read(); @@ -216,7 +218,9 @@ public void onBatchEmission( return; } - LOG.info("Emission timer: " + processingState); + if(LOG.isTraceEnabled()) { + LOG.trace("Emission timer: " + processingState); + } this.numberOfResultsBeforeBundleStart = processingState.getResultCount(); diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java index 16381c470183..96847d8b4790 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java @@ -155,7 +155,7 @@ public void onBundleFinish() { if (processingState.isNextEvent(currentSequence)) { // Event matches expected sequence state = currentStateState.read(); - if(state == null) { + if (state == null) { LOG.warn("Unexpectedly got an empty state. Most likely cause is pipeline drainage."); return null; } @@ -191,7 +191,6 @@ public void onBundleFinish() { } - protected void saveStates( ValueState> processingStatusState, ProcessingState processingStatus, @@ -291,13 +290,14 @@ private void bufferEvent(long currentSequence, EventTypeT currentEvent, abstract boolean checkForSequenceGapInBufferedEvents(); - @Nullable StateTypeT processBufferedEventRange(ProcessingState processingState, + @Nullable + StateTypeT processBufferedEventRange(ProcessingState processingState, @Nullable StateTypeT state, OrderedListState bufferedEventsState, MultiOutputReceiver outputReceiver, Timer largeBatchEmissionTimer, CompletedSequenceRange completedSequenceRange) { Long earliestBufferedSequence = processingState.getEarliestBufferedSequence(); Long latestBufferedSequence = processingState.getLatestBufferedSequence(); - if(earliestBufferedSequence == null || latestBufferedSequence == null) { + if (earliestBufferedSequence == null || latestBufferedSequence == null) { return state; } Instant startRange = fromLong(earliestBufferedSequence); @@ -316,7 +316,20 @@ private void bufferEvent(long currentSequence, EventTypeT currentEvent, long eventSequence = eventTimestamp.getMillis(); EventTypeT bufferedEvent = timestampedEvent.getValue(); + boolean skipProcessing = false; + + if (completedSequenceRange != null && eventSequence < completedSequenceRange.getStart()) { + // In case of global sequence processing - remove the elements below the range start + skipProcessing = true; + endClearRange = fromLong(eventSequence); + } if (processingState.checkForDuplicateBatchedEvent(eventSequence)) { + // There could be multiple events under the same sequence number. Only the first one + // will get processed. The rest are considered duplicates. + skipProcessing = true; + } + + if(skipProcessing) { outputReceiver .get(unprocessedEventsTupleTag) .output( @@ -325,15 +338,19 @@ private void bufferEvent(long currentSequence, EventTypeT currentEvent, KV.of( eventSequence, UnprocessedEvent.create(bufferedEvent, Reason.duplicate)))); + // TODO: When there is a large number of duplicates this can cause a situation where + // we produce too much output and the runner will start throwing unrecoverable errors. + // Need to add counting logic to accumulate both the normal and DLQ outputs. continue; } Long lastOutputSequence = processingState.getLastOutputSequence(); - boolean currentEventIsNotInSequence = lastOutputSequence != null && eventSequence > lastOutputSequence + 1; + boolean currentEventIsNotInSequence = + lastOutputSequence != null && eventSequence > lastOutputSequence + 1; boolean stopProcessing = checkForSequenceGapInBufferedEvents() ? currentEventIsNotInSequence : // TODO: can it be made more clear? - (! (eventSequence <= completedSequenceRange.getEnd()) && currentEventIsNotInSequence); + (!(eventSequence <= completedSequenceRange.getEnd()) && currentEventIsNotInSequence); if (stopProcessing) { processingState.foundSequenceGap(eventSequence); // Records will be cleared up to this element @@ -352,11 +369,15 @@ private void bufferEvent(long currentSequence, EventTypeT currentEvent, endClearRange = fromLong(eventSequence + 1); try { - if(state == null) { - LOG.info("Creating a new state: " + processingState.getKey() + " " + bufferedEvent); + if (state == null) { + if(LOG.isTraceEnabled()) { + LOG.trace("Creating a new state: " + processingState.getKey() + " " + bufferedEvent); + } state = eventExaminer.createStateOnInitialEvent(bufferedEvent); } else { - LOG.info("Mutating " + processingState.getKey() + " " + bufferedEvent); + if(LOG.isTraceEnabled()) { + LOG.trace("Mutating " + processingState.getKey() + " " + bufferedEvent); + } state.mutate(bufferedEvent); } } catch (Exception e) { diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java index 12dcba6fd363..4dc9e077706c 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java @@ -121,6 +121,39 @@ public void testHandlingOfDuplicateSequences() throws CannotProvideCoderExceptio LARGE_MAX_RESULTS_PER_OUTPUT); } + @Test + public void testTreatingSequencesBelowInitialAsDuplicates() throws CannotProvideCoderException { + Event[] events = { + Event.create(3, "id-1", "d"), + Event.create(2, "id-1", "c"), + + // Earlier events + Event.create(-1, "id-1", "early-1"), + Event.create(-2, "id-1", "early-2"), + + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b") + }; + + Collection> expectedOutput = new ArrayList<>(); + expectedOutput.add(KV.of("id-1", "a")); + expectedOutput.add(KV.of("id-1", "ab")); + expectedOutput.add(KV.of("id-1", "abc")); + expectedOutput.add(KV.of("id-1", "abcd")); + + Collection>>> duplicates = new ArrayList<>(); + duplicates.add(KV.of("id-1", KV.of(-1L, UnprocessedEvent.create("early-1", Reason.duplicate)))); + duplicates.add(KV.of("id-1", KV.of(-2L, UnprocessedEvent.create("early-2", Reason.duplicate)))); + + testGlobalSequenceProcessing( + events, + expectedOutput, + duplicates, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT); + } + @Test public void testHandlingOfCheckedExceptions() throws CannotProvideCoderException { Event[] events = { From 29f36bbaa7560e068256e56fae90449c5edf0353 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Wed, 11 Sep 2024 12:54:52 -0700 Subject: [PATCH 12/33] Added providing a global sequence combiner through a handler. --- .../ordered/GlobalSequenceTracker.java | 14 +++-- .../ordered/OrderedEventProcessor.java | 2 +- .../ordered/OrderedProcessingHandler.java | 54 ++++++++++++------- .../StringBufferOrderedProcessingHandler.java | 2 - 4 files changed, 46 insertions(+), 26 deletions(-) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java index 55437749e759..d9547ef766d1 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java @@ -1,7 +1,6 @@ package org.apache.beam.sdk.extensions.ordered; import org.apache.beam.sdk.extensions.ordered.CompletedSequenceRange.CompletedSequenceRangeCoder; -import org.apache.beam.sdk.extensions.ordered.combiner.DefaultSequenceCombiner; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.AfterFirst; @@ -14,15 +13,19 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; +import org.checkerframework.checker.initialization.qual.Initialized; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.UnknownKeyFor; import org.joda.time.Duration; class GlobalSequenceTracker> extends PTransform>>>, PCollectionView> { - private final DefaultSequenceCombiner sequenceCombiner; + private final Combine.GloballyAsSingletonView>>, CompletedSequenceRange> sideInputProducer; - public GlobalSequenceTracker(EventExaminer eventExaminer) { - this.sequenceCombiner = new DefaultSequenceCombiner<>(eventExaminer); + public GlobalSequenceTracker( + Combine.GloballyAsSingletonView>>, CompletedSequenceRange> sideInputProducer) { + this.sideInputProducer = sideInputProducer; } @Override @@ -31,6 +34,7 @@ public PCollectionView expand( input.getPipeline().getCoderRegistry().registerCoderForClass( CompletedSequenceRange.class, CompletedSequenceRangeCoder.of()); + return input // TODO: get the windowing strategy from the input rather than assume global windows. @@ -43,6 +47,6 @@ public PCollectionView expand( AfterPane.elementCountAtLeast(1), AfterProcessingTime.pastFirstElementInPane() .plusDelayOf(Duration.standardSeconds(5)))))) - .apply("Create Side Input", Combine.globally(sequenceCombiner).asSingletonView()); + .apply("Create Side Input", sideInputProducer); } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java index 7abb92e75582..edda81aa9150 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java @@ -131,7 +131,7 @@ public OrderedEventProcessorResult expand( .apply("Convert to SequenceAndTimestamp", ParDo.of(new ToTimestampedEventConverter<>())) .apply("Global Sequence Tracker", - new GlobalSequenceTracker<>(handler.getEventExaminer())); + new GlobalSequenceTracker<>(handler.getGlobalSequenceCombiner())); PCollection>> tickers = input.apply("Create Tickers", new PerKeyTickerGenerator<>(keyCoder, eventCoder, Duration.standardSeconds(5))); diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java index 997e2910278e..be6b5ad2b8ac 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java @@ -22,21 +22,27 @@ import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.extensions.ordered.combiner.DefaultSequenceCombiner; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.Combine.GloballyAsSingletonView; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TimestampedValue; +import org.checkerframework.checker.initialization.qual.Initialized; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; +import org.checkerframework.checker.nullness.qual.UnknownKeyFor; import org.joda.time.Duration; /** * Parent class for Ordered Processing configuration handlers. * - * @param type of events to be processed - * @param type of keys which will be used to group the events - * @param type of internal State which will be used for processing + * @param type of events to be processed + * @param type of keys which will be used to group the events + * @param type of internal State which will be used for processing * @param type of the result of the processing which will be output */ public abstract class OrderedProcessingHandler< - EventT, KeyT, StateT extends MutableState, ResultT> + EventT, KeyT, StateT extends MutableState, ResultT> implements Serializable { private static final int DEFAULT_STATUS_UPDATE_FREQUENCY_SECONDS = 5; @@ -54,14 +60,14 @@ public abstract class OrderedProcessingHandler< private boolean produceStatusUpdateOnEveryEvent = DEFAULT_PRODUCE_STATUS_UPDATE_ON_EVERY_EVENT; private SequenceType sequenceType = SequenceType.PER_KEY; - private @Nullable Long initialGlobalSequence; + private @Nullable GloballyAsSingletonView>>, CompletedSequenceRange> globalSequenceCombiner; /** * Provide concrete classes which will be used by the ordered processing transform. * - * @param eventTClass class of the events - * @param keyTClass class of the keys - * @param stateTClass class of the state + * @param eventTClass class of the events + * @param keyTClass class of the keys + * @param stateTClass class of the state * @param resultTClass class of the results */ public OrderedProcessingHandler( @@ -73,10 +79,12 @@ public OrderedProcessingHandler( this.keyTClass = keyTClass; this.stateTClass = stateTClass; this.resultTClass = resultTClass; - this.initialGlobalSequence = null; + this.globalSequenceCombiner = null; } - /** @return the event examiner instance which will be used by the transform. */ + /** + * @return the event examiner instance which will be used by the transform. + */ public abstract @NonNull EventExaminer getEventExaminer(); /** @@ -86,11 +94,11 @@ public OrderedProcessingHandler( * PCollection. If the input PCollection doesn't use KVCoder, it will attempt to get the coder * from the pipeline's coder registry. * - * @param pipeline of the transform + * @param pipeline of the transform * @param inputCoder input coder of the transform * @return event coder * @throws CannotProvideCoderException if the method can't determine the coder based on the above - * algorithm. + * algorithm. */ public @NonNull Coder getEventCoder( Pipeline pipeline, Coder>> inputCoder) @@ -130,7 +138,7 @@ public OrderedProcessingHandler( * @param inputCoder * @return * @throws CannotProvideCoderException if the method can't determine the coder based on the above - * algorithm. + * algorithm. */ public @NonNull Coder getKeyCoder( Pipeline pipeline, Coder>> inputCoder) @@ -161,7 +169,7 @@ public OrderedProcessingHandler( *

Default is 5 seconds. * * @return the frequency of updates. If null is returned, no updates will be emitted on a - * scheduled basis. + * scheduled basis. */ public @Nullable Duration getStatusUpdateFrequency() { return statusUpdateFrequency; @@ -230,11 +238,21 @@ public void setSequenceType(SequenceType sequenceType) { this.sequenceType = sequenceType; } - public @javax.annotation.Nullable Long getInitialGlobalSequence() { - return initialGlobalSequence; + public GloballyAsSingletonView>>, CompletedSequenceRange> getGlobalSequenceCombiner() { + if (getSequenceType() != SequenceType.GLOBAL) { + throw new IllegalStateException( + "Global Sequence Combiner is only useful when the sequence is global. Current sequence type: " + + getSequenceType()); + } + if (globalSequenceCombiner == null) { + globalSequenceCombiner = Combine.globally( + new DefaultSequenceCombiner(getEventExaminer())).asSingletonView(); + } + return globalSequenceCombiner; } - public void setInitialGlobalSequence(Long initialGlobalSequence) { - this.initialGlobalSequence = initialGlobalSequence; + public void setGlobalSequenceCombiner( + GloballyAsSingletonView>>, CompletedSequenceRange> globalSequenceCombiner) { + this.globalSequenceCombiner = globalSequenceCombiner; } } diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/StringBufferOrderedProcessingHandler.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/StringBufferOrderedProcessingHandler.java index 95328981a10e..72f3a3cf21b6 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/StringBufferOrderedProcessingHandler.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/StringBufferOrderedProcessingHandler.java @@ -32,8 +32,6 @@ public class StringBufferOrderedProcessingHandler public StringBufferOrderedProcessingHandler(int emissionFrequency, long initialSequence) { super(String.class, String.class, StringBuilderState.class, String.class); this.eventExaminer = new StringEventExaminer(initialSequence, emissionFrequency); - // needed when testing global sequences - setInitialGlobalSequence(initialSequence); } @Override From 464979531e7f3a60b93c215be6908ab991311570 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Wed, 11 Sep 2024 14:27:15 -0700 Subject: [PATCH 13/33] Added SequenceRangeAccumulatorCoder and tests. Improved logic of creating timers. --- .../ordered/GlobalSequencesProcessorDoFn.java | 8 +- .../extensions/ordered/ProcessingState.java | 7 ++ .../sdk/extensions/ordered/ProcessorDoFn.java | 4 +- .../combiner/SequenceRangeAccumulator.java | 107 ++++++++++++------ .../SequenceRangeAccumulatorCoderTest.java | 56 +++++++++ 5 files changed, 145 insertions(+), 37 deletions(-) create mode 100644 sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorCoderTest.java diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java index 657d37515c5a..5ba3abb75ce3 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java @@ -122,7 +122,7 @@ public void processElement(ProcessContext context, EventKeyT key = eventAndSequence.getKey(); long sequence = eventAndSequence.getValue().getKey(); - if(LOG.isTraceEnabled()) { + if (LOG.isTraceEnabled()) { LOG.trace(key + ": " + sequence + " lastSequence: " + lastContinuousSequence); } @@ -145,7 +145,6 @@ public void processElement(ProcessContext context, // TODO: we can keep resetting this into the future under heavy load. // Need to add logic to avoid doing it. - // setBatchEmissionTimerIfNeeded(batchEmissionTimer, processingState); return; @@ -182,7 +181,8 @@ public void processElement(ProcessContext context, private void setBatchEmissionTimerIfNeeded(Timer batchEmissionTimer, ProcessingState processingState) { CompletedSequenceRange lastCompleteGlobalSequence = processingState.getLastCompleteGlobalSequence(); - if (processingState.getBufferedEventCount() > 0 && lastCompleteGlobalSequence != null) { + if (lastCompleteGlobalSequence != null && + processingState.haveGloballySequencedEventsToBeProcessed()) { batchEmissionTimer.set(lastCompleteGlobalSequence.getTimestamp()); } } @@ -218,7 +218,7 @@ public void onBatchEmission( return; } - if(LOG.isTraceEnabled()) { + if (LOG.isTraceEnabled()) { LOG.trace("Emission timer: " + processingState); } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java index 120aca8cdaed..2a1607beaebc 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java @@ -305,6 +305,13 @@ public void updateGlobalSequenceDetails(CompletedSequenceRange updated) { this.lastCompleteGlobalSequence = updated; } + public boolean haveGloballySequencedEventsToBeProcessed() { + return bufferedEventCount > 0 + && lastCompleteGlobalSequence != null + && earliestBufferedSequence != null && + earliestBufferedSequence <= lastCompleteGlobalSequence.getEnd(); + } + /** * Coder for the processing status. * diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java index 96847d8b4790..af21efff921d 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java @@ -241,7 +241,9 @@ protected void emitProcessingStatus( ProcessingState processingState, MultiOutputReceiver outputReceiver, Instant statusTimestamp) { - LOG.info("Emitting status for: " + processingState.getKey() + ", " + processingState); + if(LOG.isTraceEnabled()) { + LOG.trace("Emitting status for: " + processingState.getKey() + ", " + processingState); + } outputReceiver .get(statusTupleTag) .outputWithTimestamp( diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java index 60d42a90387d..92f9d92663b3 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java @@ -10,6 +10,9 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.extensions.ordered.CompletedSequenceRange; import org.apache.commons.lang3.tuple.Pair; import org.checkerframework.checker.initialization.qual.Initialized; @@ -23,7 +26,7 @@ static Instant max(Instant a, Instant b) { return a.isAfter(b) ? a : b; } - private final TreeMap> accumulator = new TreeMap<>(); + private final TreeMap> data = new TreeMap<>(); private @Nullable Long initialSequence = null; public void add(long sequence, Instant timestamp, boolean isInitialSequence) { @@ -41,11 +44,10 @@ public void add(long sequence, Instant timestamp, boolean isInitialSequence) { return; } - long lowerBound = sequence; - long upperBound = sequence; + long lowerBound = sequence, upperBound = sequence; - Entry> lowerRange = accumulator.floorEntry(sequence); - if(lowerRange != null) { + Entry> lowerRange = data.floorEntry(sequence); + if (lowerRange != null) { long inclusiveUpperBoundary = lowerRange.getValue().getLeft(); if (sequence <= inclusiveUpperBoundary) { // Duplicate. No need to adjust the timestamp. @@ -53,37 +55,38 @@ public void add(long sequence, Instant timestamp, boolean isInitialSequence) { } if (inclusiveUpperBoundary + 1 == sequence) { - // The new element extends this range + // The new element extends the lower range. Remove the range. timestamp = max(timestamp, lowerRange.getValue().getValue()); lowerBound = lowerRange.getKey(); - accumulator.remove(lowerRange.getKey()); + data.remove(lowerRange.getKey()); } } long nextSequenceNumber = sequence + 1; - Pair upperRange = accumulator.get(nextSequenceNumber); + Pair upperRange = data.get(nextSequenceNumber); if (upperRange != null) { + // The new element will extend the upper range. Remove the range. timestamp = max(timestamp, upperRange.getRight()); upperBound = upperRange.getLeft(); - accumulator.remove(nextSequenceNumber); + data.remove(nextSequenceNumber); } - accumulator.put(lowerBound, Pair.of(upperBound, timestamp)); + data.put(lowerBound, Pair.of(upperBound, timestamp)); } private void clearRangesBelowInitialSequence(long sequence, Instant timestamp) { // First, adjust the current range, if any - Entry> lowerRange = accumulator.floorEntry(sequence); - if(lowerRange != null + Entry> lowerRange = data.floorEntry(sequence); + if (lowerRange != null && lowerRange.getKey() < sequence && lowerRange.getValue().getLeft() > sequence) { // The sequence is in the middle of the range. Adjust it. - accumulator.remove(lowerRange.getKey()); - accumulator.put(sequence, + data.remove(lowerRange.getKey()); + data.put(sequence, Pair.of(lowerRange.getValue().getKey(), max(timestamp, lowerRange.getValue() - .getValue()))); + .getValue()))); } - accumulator.subMap(Long.MIN_VALUE, sequence).clear(); + data.subMap(Long.MIN_VALUE, sequence).clear(); } public CompletedSequenceRange largestContinuousRange() { @@ -91,9 +94,9 @@ public CompletedSequenceRange largestContinuousRange() { return CompletedSequenceRange.EMPTY; } - Entry> firstEntry = accumulator.firstEntry(); + Entry> firstEntry = data.firstEntry(); if (firstEntry == null) { - throw new IllegalStateException("First entry is null"); + throw new IllegalStateException("First entry is null when initial sequence is set."); } Long start = firstEntry.getKey(); Long end = firstEntry.getValue().getLeft(); @@ -102,7 +105,7 @@ public CompletedSequenceRange largestContinuousRange() { } public int numberOfRanges() { - return accumulator.size(); + return data.size(); } @@ -116,29 +119,29 @@ public void merge(SequenceRangeAccumulator another) { if (another.initialSequence != null) { long newInitialSequence = another.initialSequence; this.initialSequence = newInitialSequence; - Entry> firstEntry = another.accumulator.firstEntry(); - if(firstEntry != null) { + Entry> firstEntry = another.data.firstEntry(); + if (firstEntry != null) { Instant timestampOfTheInitialRange = firstEntry.getValue().getRight(); clearRangesBelowInitialSequence(newInitialSequence, timestampOfTheInitialRange); } } - another.accumulator.entrySet().stream().forEach( + another.data.entrySet().forEach( entry -> { long lowerBound = entry.getKey(); long upperBound = entry.getValue().getLeft(); - if(this.initialSequence != null) { - if(upperBound < initialSequence) { + if (this.initialSequence != null) { + if (upperBound < initialSequence) { // The whole range is below the initial sequence. Ignore it. return; } - if(lowerBound < initialSequence) { + if (lowerBound < initialSequence) { // This will cause pruning of the range up to the initial sequence lowerBound = this.initialSequence; } } - Entry> lowerRange = this.accumulator.floorEntry(lowerBound); + Entry> lowerRange = this.data.floorEntry(lowerBound); if (lowerRange != null) { if (lowerRange.getValue().getLeft() < lowerBound - 1) { @@ -150,7 +153,7 @@ public void merge(SequenceRangeAccumulator another) { } } - Entry> upperRange = this.accumulator.floorEntry(upperBound + 1); + Entry> upperRange = this.data.floorEntry(upperBound + 1); if (upperRange == null || (lowerRange != null && Objects.equals(upperRange.getKey(), lowerRange.getKey()))) { // Nothing to do - either there is no adjacent upper range or it equals the lower range @@ -161,14 +164,14 @@ public void merge(SequenceRangeAccumulator another) { Instant latestTimestamp = removeAllRanges(lowerBound, upperBound, entry.getValue().getRight()); - this.accumulator.put(lowerBound, Pair.of(upperBound, latestTimestamp)); + this.data.put(lowerBound, Pair.of(upperBound, latestTimestamp)); } ); } private Instant removeAllRanges(long lowerBound, long upperBound, Instant currentTimestamp) { Instant result = currentTimestamp; - SortedMap> rangesToRemove = accumulator.subMap(lowerBound, + SortedMap> rangesToRemove = data.subMap(lowerBound, upperBound); for (Pair value : rangesToRemove.values()) { result = result.isAfter(value.getRight()) ? result : value.getRight(); @@ -177,21 +180,61 @@ private Instant removeAllRanges(long lowerBound, long upperBound, Instant curren return result; } + @Override + public boolean equals(@Nullable Object o) { + if (this == o) { + return true; + } + if (!(o instanceof SequenceRangeAccumulator)) { + return false; + } + SequenceRangeAccumulator that = (SequenceRangeAccumulator) o; + return data.equals(that.data) && Objects.equals(initialSequence, that.initialSequence); + } + + @Override + public int hashCode() { + return Objects.hash(data, initialSequence); + } + + @Override + public String toString() { + return "SequenceRangeAccumulator{initialSequence=" + initialSequence + ", data=" + data + '}'; + } + public static class SequenceRangeAccumulatorCoder extends CustomCoder { - // TODO implement + private final NullableCoder initialSequenceCoder = NullableCoder.of(VarLongCoder.of()); + private final VarIntCoder numberOfRangesCoder = VarIntCoder.of(); + private final VarLongCoder dataCoder = VarLongCoder.of(); + @Override public void encode(SequenceRangeAccumulator value, @UnknownKeyFor @NonNull @Initialized OutputStream outStream) throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { - + numberOfRangesCoder.encode(value.numberOfRanges(), outStream); + initialSequenceCoder.encode(value.initialSequence, outStream); + for (Entry> entry : value.data.entrySet()) { + dataCoder.encode(entry.getKey(), outStream); + dataCoder.encode(entry.getValue().getLeft(), outStream); + dataCoder.encode(entry.getValue().getRight().getMillis(), outStream); + } } @Override public SequenceRangeAccumulator decode( @UnknownKeyFor @NonNull @Initialized InputStream inStream) throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { - return new SequenceRangeAccumulator(); + SequenceRangeAccumulator result = new SequenceRangeAccumulator(); + int numberOfRanges = numberOfRangesCoder.decode(inStream); + result.initialSequence = initialSequenceCoder.decode(inStream); + for (int i = 0; i < numberOfRanges; i++) { + long key = dataCoder.decode(inStream); + long upperBound = dataCoder.decode(inStream); + long millis = dataCoder.decode(inStream); + result.data.put(key, Pair.of(upperBound, Instant.ofEpochMilli(millis))); + } + return result; } } } diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorCoderTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorCoderTest.java new file mode 100644 index 000000000000..7cd3fa8ed613 --- /dev/null +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorCoderTest.java @@ -0,0 +1,56 @@ +package org.apache.beam.sdk.extensions.ordered.combiner; + +import static org.junit.Assert.assertEquals; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.beam.sdk.extensions.ordered.combiner.SequenceRangeAccumulator.SequenceRangeAccumulatorCoder; +import org.joda.time.Instant; +import org.junit.Test; + +public class SequenceRangeAccumulatorCoderTest { + + private SequenceRangeAccumulatorCoder coder = new SequenceRangeAccumulatorCoder(); + + @Test + public void testEncodingEmptyAccumulator() throws IOException { + SequenceRangeAccumulator empty = new SequenceRangeAccumulator(); + + doTestEncodingAndDecoding(empty); + } + + @Test + public void testEncodingAccumulatorWithoutInitialSequence() throws IOException { + SequenceRangeAccumulator accumulator = new SequenceRangeAccumulator(); + accumulator.add(1, Instant.now(), false); + accumulator.add(2, Instant.now(), false); + accumulator.add(3, Instant.now(), false); + accumulator.add(5, Instant.now(), false); + accumulator.add(6, Instant.now(), false); + + doTestEncodingAndDecoding(accumulator); + } + + @Test + public void testEncodingAccumulatorWithInitialSequence() throws IOException { + SequenceRangeAccumulator accumulator = new SequenceRangeAccumulator(); + accumulator.add(1, Instant.now(), true); + accumulator.add(2, Instant.now(), false); + accumulator.add(3, Instant.now(), false); + accumulator.add(5, Instant.now(), false); + accumulator.add(6, Instant.now(), false); + + doTestEncodingAndDecoding(accumulator); + } + + private void doTestEncodingAndDecoding(SequenceRangeAccumulator value) + throws IOException { + ByteArrayOutputStream output = new ByteArrayOutputStream(); + coder.encode(value, output); + + SequenceRangeAccumulator decoded = coder.decode(new ByteArrayInputStream(output.toByteArray())); + assertEquals("Accumulator", value, decoded); + } + +} From 962bfdd3539d04f435317f054f2e91a554e3e20f Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Wed, 11 Sep 2024 15:24:14 -0700 Subject: [PATCH 14/33] Fixed logging levels (moved them to "trace") on several transforms. --- .../beam/sdk/extensions/ordered/ProcessorDoFn.java | 10 ++++++---- .../ordered/SequencePerKeyProcessorDoFn.java | 4 ++-- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java index af21efff921d..3f23efc440a1 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java @@ -270,10 +270,12 @@ protected boolean reachedMaxResultCountForBundle( : numberOfResultsBeforeBundleStart.longValue()) >= maxNumberOfResultsToProduce; if (exceeded) { - LOG.info( - "Setting the timer to output next batch of events for key '" - + processingState.getKey() - + "'"); + if(LOG.isTraceEnabled()) { + LOG.trace( + "Setting the timer to output next batch of events for key '" + + processingState.getKey() + + "'"); + } // See GroupIntoBatches for examples on how to hold the timestamp. // TODO: test that on draining the pipeline all the results are still produced correctly. // See: https://github.com/apache/beam/issues/30781 diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java index 534cccae0001..f47f667718f5 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java @@ -151,8 +151,8 @@ public void processElement( private boolean checkIfProcessingIsCompleted(ProcessingState processingState) { boolean result = processingState.isProcessingCompleted(); - if (result) { - LOG.info("Processing for key '" + processingState.getKey() + "' is completed."); + if (result && LOG.isTraceEnabled()) { + LOG.trace("Processing for key '" + processingState.getKey() + "' is completed."); } return result; } From d0a7a14f022991a113aae8260b0b2c4445972158 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Thu, 12 Sep 2024 11:44:54 -0700 Subject: [PATCH 15/33] Round of code improvements and cleanups. --- .../ordered/GlobalSequencesProcessorDoFn.java | 28 ++-- .../extensions/ordered/ProcessingState.java | 8 +- .../sdk/extensions/ordered/ProcessorDoFn.java | 134 ++++++++---------- .../combiner/SequenceRangeAccumulator.java | 2 +- .../SequenceRangeAccumulatorTest.java | 68 ++++++++- 5 files changed, 143 insertions(+), 97 deletions(-) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java index 5ba3abb75ce3..ac35ca21a48b 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java @@ -105,10 +105,10 @@ boolean checkForSequenceGapInBufferedEvents() { @ProcessElement public void processElement(ProcessContext context, @Element KV> eventAndSequence, - @StateId(BUFFERED_EVENTS) OrderedListState bufferedEventsState, + @StateId(BUFFERED_EVENTS) OrderedListState bufferedEventsProxy, @AlwaysFetched @StateId(PROCESSING_STATE) - ValueState> processingStateState, - @StateId(MUTABLE_STATE) ValueState mutableStateState, + ValueState> processingStateProxy, + @StateId(MUTABLE_STATE) ValueState mutableStateProxy, @TimerId(STATUS_EMISSION_TIMER) Timer statusEmissionTimer, @TimerId(BATCH_EMISSION_TIMER) Timer batchEmissionTimer, MultiOutputReceiver outputReceiver, @@ -118,7 +118,6 @@ public void processElement(ProcessContext context, latestContinuousSequenceSideInput); EventT event = eventAndSequence.getValue().getValue(); - EventKeyT key = eventAndSequence.getKey(); long sequence = eventAndSequence.getValue().getKey(); @@ -126,7 +125,7 @@ public void processElement(ProcessContext context, LOG.trace(key + ": " + sequence + " lastSequence: " + lastContinuousSequence); } - ProcessingState processingState = processingStateState.read(); + ProcessingState processingState = processingStateProxy.read(); if (processingState == null) { // This is the first time we see this key/window pair @@ -140,11 +139,9 @@ public void processElement(ProcessContext context, processingState.updateGlobalSequenceDetails(lastContinuousSequence); if (event == null) { - // This is the ticker event. We only need to update the state as it relates to the global sequence. - processingStateState.write(processingState); + // This is a ticker event. We only need to update the state as it relates to the global sequence. + processingStateProxy.write(processingState); - // TODO: we can keep resetting this into the future under heavy load. - // Need to add logic to avoid doing it. setBatchEmissionTimerIfNeeded(batchEmissionTimer, processingState); return; @@ -162,27 +159,26 @@ public void processElement(ProcessContext context, sequence, event, processingState, - mutableStateState, - bufferedEventsState, + mutableStateProxy, + bufferedEventsProxy, outputReceiver); saveStates( - processingStateState, + processingStateProxy, processingState, - mutableStateState, + mutableStateProxy, state, outputReceiver, window.maxTimestamp()); - // Only if the record matches the sequence it can be output now - // TODO: refactor the code from SequencePerKeyDoFn + setBatchEmissionTimerIfNeeded(batchEmissionTimer, processingState); } private void setBatchEmissionTimerIfNeeded(Timer batchEmissionTimer, ProcessingState processingState) { CompletedSequenceRange lastCompleteGlobalSequence = processingState.getLastCompleteGlobalSequence(); if (lastCompleteGlobalSequence != null && - processingState.haveGloballySequencedEventsToBeProcessed()) { + processingState.thereAreGloballySequencedEventsToBeProcessed()) { batchEmissionTimer.set(lastCompleteGlobalSequence.getTimestamp()); } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java index 2a1607beaebc..fc41164cedf6 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java @@ -302,10 +302,16 @@ public long resultsProducedInBundle(long numberOfResultsBeforeBundleStart) { } public void updateGlobalSequenceDetails(CompletedSequenceRange updated) { + if(thereAreGloballySequencedEventsToBeProcessed()) { + // We don't update the timer if we can already process events in the onTimer batch. + // Otherwise, it's possible that we will be pushing the timer to later timestamps + // without a chance to run and produce output. + return; + } this.lastCompleteGlobalSequence = updated; } - public boolean haveGloballySequencedEventsToBeProcessed() { + public boolean thereAreGloballySequencedEventsToBeProcessed() { return bufferedEventCount > 0 && lastCompleteGlobalSequence != null && earliestBufferedSequence != null && diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java index 3f23efc440a1..a596bd00fb1f 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java @@ -16,18 +16,15 @@ import org.slf4j.LoggerFactory; /** - * Main DoFn for processing ordered events. + * Base DoFn for processing ordered events. * - * @param - * @param - * @param + * @param type of the events to process + * @param event key type + * @param state type */ -abstract class ProcessorDoFn< - EventTypeT, - EventKeyTypeT, - ResultTypeT, - StateTypeT extends MutableState> - extends DoFn>, KV> { +abstract class ProcessorDoFn> + extends DoFn>, KV> { private static final Logger LOG = LoggerFactory.getLogger(ProcessorDoFn.class); @@ -36,38 +33,27 @@ abstract class ProcessorDoFn< protected static final String STATUS_EMISSION_TIMER = "statusTimer"; protected static final String WINDOW_CLOSED = "windowClosed"; - protected final EventExaminer eventExaminer; + protected final EventExaminer eventExaminer; - private final TupleTag> statusTupleTag; + private final TupleTag> statusTupleTag; protected final Duration statusUpdateFrequency; - protected final TupleTag> mainOutputTupleTag; - protected final TupleTag>>> + protected final TupleTag> mainOutputTupleTag; + protected final TupleTag>>> unprocessedEventsTupleTag; private final boolean produceStatusUpdateOnEveryEvent; private final long maxNumberOfResultsToProduce; - protected @Nullable Long numberOfResultsBeforeBundleStart = Long.valueOf(0); + protected @Nullable Long numberOfResultsBeforeBundleStart = 0L; - /** - * Stateful DoFn to do the bulk of processing. - * - * @param eventExaminer - * @param mainOutputTupleTag - * @param statusTupleTag - * @param statusUpdateFrequency - * @param unprocessedEventTupleTag - * @param produceStatusUpdateOnEveryEvent - * @param maxNumberOfResultsToProduce - */ ProcessorDoFn( - EventExaminer eventExaminer, - TupleTag> mainOutputTupleTag, - TupleTag> statusTupleTag, + EventExaminer eventExaminer, + TupleTag> mainOutputTupleTag, + TupleTag> statusTupleTag, Duration statusUpdateFrequency, - TupleTag>>> + TupleTag>>> unprocessedEventTupleTag, boolean produceStatusUpdateOnEveryEvent, long maxNumberOfResultsToProduce) { @@ -92,6 +78,9 @@ public void onBundleFinish() { numberOfResultsBeforeBundleStart = null; } + /** + * @return true if each event needs to be examined. + */ abstract boolean checkForInitialEvent(); /** @@ -99,12 +88,12 @@ public void onBundleFinish() { * * @return newly created or updated State. If null is returned - the event wasn't processed. */ - protected @javax.annotation.Nullable StateTypeT processNewEvent( + protected @javax.annotation.Nullable StateT processNewEvent( long currentSequence, - EventTypeT currentEvent, - ProcessingState processingState, - ValueState currentStateState, - OrderedListState bufferedEventsState, + EventT currentEvent, + ProcessingState processingState, + ValueState currentStateState, + OrderedListState bufferedEventsState, MultiOutputReceiver outputReceiver) { if (currentSequence == Long.MAX_VALUE) { // OrderedListState can't handle the timestamp based on MAX_VALUE. @@ -132,7 +121,7 @@ public void onBundleFinish() { return null; } - StateTypeT state; + StateT state; boolean thisIsTheLastEvent = eventExaminer.isLastEvent(currentSequence, currentEvent); if (checkForInitialEvent() && eventExaminer.isInitialEvent(currentSequence, currentEvent)) { // First event of the key/window @@ -142,7 +131,7 @@ public void onBundleFinish() { processingState.eventAccepted(currentSequence, thisIsTheLastEvent); - ResultTypeT result = state.produceResult(); + ResultT result = state.produceResult(); if (result != null) { outputReceiver.get(mainOutputTupleTag).output(KV.of(processingState.getKey(), result)); processingState.resultProduced(); @@ -172,7 +161,7 @@ public void onBundleFinish() { return null; } - ResultTypeT result = state.produceResult(); + ResultT result = state.produceResult(); if (result != null) { outputReceiver.get(mainOutputTupleTag).output(KV.of(processingState.getKey(), result)); processingState.resultProduced(); @@ -186,16 +175,15 @@ public void onBundleFinish() { bufferEvent(currentSequence, currentEvent, processingState, bufferedEventsState, thisIsTheLastEvent); - // This will signal that the state hasn't been mutated and we don't need to save it. + // This will signal that the state hasn't been mutated. We don't need to save it. return null; } - protected void saveStates( - ValueState> processingStatusState, - ProcessingState processingStatus, - ValueState currentStateState, - @Nullable StateTypeT state, + ValueState> processingStatusState, + ProcessingState processingStatus, + ValueState currentStateState, + @Nullable StateT state, MultiOutputReceiver outputReceiver, Instant windowTimestamp) { // There is always a change to the processing status @@ -217,11 +205,11 @@ protected void saveStates( void processStatusTimerEvent(MultiOutputReceiver outputReceiver, Timer statusEmissionTimer, ValueState windowClosedState, - ValueState> processingStateState) { - ProcessingState currentState = processingStateState.read(); + ValueState> processingStateState) { + ProcessingState currentState = processingStateState.read(); if (currentState == null) { // This could happen if the state has been purged already during the draining. - // It means that there is nothing that we can do and we just need to return. + // It means that there is nothing that we can do. LOG.warn( "Current processing state is null in onStatusEmission() - most likely the pipeline is shutting down."); return; @@ -238,10 +226,10 @@ void processStatusTimerEvent(MultiOutputReceiver outputReceiver, Timer statusEmi } protected void emitProcessingStatus( - ProcessingState processingState, + ProcessingState processingState, MultiOutputReceiver outputReceiver, Instant statusTimestamp) { - if(LOG.isTraceEnabled()) { + if (LOG.isTraceEnabled()) { LOG.trace("Emitting status for: " + processingState.getKey() + ", " + processingState); } outputReceiver @@ -263,14 +251,13 @@ protected void emitProcessingStatus( protected boolean reachedMaxResultCountForBundle( - ProcessingState processingState, Timer largeBatchEmissionTimer) { + ProcessingState processingState, Timer largeBatchEmissionTimer) { boolean exceeded = processingState.resultsProducedInBundle( - numberOfResultsBeforeBundleStart == null ? 0 - : numberOfResultsBeforeBundleStart.longValue()) + numberOfResultsBeforeBundleStart == null ? 0 : numberOfResultsBeforeBundleStart) >= maxNumberOfResultsToProduce; if (exceeded) { - if(LOG.isTraceEnabled()) { + if (LOG.isTraceEnabled()) { LOG.trace( "Setting the timer to output next batch of events for key '" + processingState.getKey() @@ -284,9 +271,9 @@ protected boolean reachedMaxResultCountForBundle( return exceeded; } - private void bufferEvent(long currentSequence, EventTypeT currentEvent, - ProcessingState processingState, - OrderedListState bufferedEventsState, boolean thisIsTheLastEvent) { + private void bufferEvent(long currentSequence, EventT currentEvent, + ProcessingState processingState, + OrderedListState bufferedEventsState, boolean thisIsTheLastEvent) { Instant eventTimestamp = fromLong(currentSequence); bufferedEventsState.add(TimestampedValue.of(currentEvent, eventTimestamp)); processingState.eventBuffered(currentSequence, thisIsTheLastEvent); @@ -295,9 +282,9 @@ private void bufferEvent(long currentSequence, EventTypeT currentEvent, abstract boolean checkForSequenceGapInBufferedEvents(); @Nullable - StateTypeT processBufferedEventRange(ProcessingState processingState, - @Nullable StateTypeT state, - OrderedListState bufferedEventsState, MultiOutputReceiver outputReceiver, + StateT processBufferedEventRange(ProcessingState processingState, + @Nullable StateT state, + OrderedListState bufferedEventsState, MultiOutputReceiver outputReceiver, Timer largeBatchEmissionTimer, CompletedSequenceRange completedSequenceRange) { Long earliestBufferedSequence = processingState.getEarliestBufferedSequence(); Long latestBufferedSequence = processingState.getLatestBufferedSequence(); @@ -308,18 +295,18 @@ StateTypeT processBufferedEventRange(ProcessingState processingSt Instant endRange = fromLong(latestBufferedSequence + 1); // readRange is efficiently implemented and will bring records in batches - Iterable> events = + Iterable> events = bufferedEventsState.readRange(startRange, endRange); Instant endClearRange = startRange; // it will get re-adjusted later. - Iterator> bufferedEventsIterator = events.iterator(); + Iterator> bufferedEventsIterator = events.iterator(); while (bufferedEventsIterator.hasNext()) { - TimestampedValue timestampedEvent = bufferedEventsIterator.next(); + TimestampedValue timestampedEvent = bufferedEventsIterator.next(); Instant eventTimestamp = timestampedEvent.getTimestamp(); long eventSequence = eventTimestamp.getMillis(); - EventTypeT bufferedEvent = timestampedEvent.getValue(); + EventT bufferedEvent = timestampedEvent.getValue(); boolean skipProcessing = false; if (completedSequenceRange != null && eventSequence < completedSequenceRange.getStart()) { @@ -333,7 +320,7 @@ StateTypeT processBufferedEventRange(ProcessingState processingSt skipProcessing = true; } - if(skipProcessing) { + if (skipProcessing) { outputReceiver .get(unprocessedEventsTupleTag) .output( @@ -349,13 +336,12 @@ StateTypeT processBufferedEventRange(ProcessingState processingSt } Long lastOutputSequence = processingState.getLastOutputSequence(); - boolean currentEventIsNotInSequence = - lastOutputSequence != null && eventSequence > lastOutputSequence + 1; - boolean stopProcessing = checkForSequenceGapInBufferedEvents() ? - currentEventIsNotInSequence : - // TODO: can it be made more clear? - (!(eventSequence <= completedSequenceRange.getEnd()) && currentEventIsNotInSequence); - if (stopProcessing) { + boolean currentEventIsNextInSequence = + lastOutputSequence != null && eventSequence == lastOutputSequence + 1; + boolean continueProcessing = checkForSequenceGapInBufferedEvents() ? + currentEventIsNextInSequence : + (eventSequence <= completedSequenceRange.getEnd() || currentEventIsNextInSequence); + if (!continueProcessing) { processingState.foundSequenceGap(eventSequence); // Records will be cleared up to this element endClearRange = fromLong(eventSequence); @@ -374,12 +360,12 @@ StateTypeT processBufferedEventRange(ProcessingState processingSt try { if (state == null) { - if(LOG.isTraceEnabled()) { + if (LOG.isTraceEnabled()) { LOG.trace("Creating a new state: " + processingState.getKey() + " " + bufferedEvent); } state = eventExaminer.createStateOnInitialEvent(bufferedEvent); } else { - if(LOG.isTraceEnabled()) { + if (LOG.isTraceEnabled()) { LOG.trace("Mutating " + processingState.getKey() + " " + bufferedEvent); } state.mutate(bufferedEvent); @@ -396,7 +382,7 @@ StateTypeT processBufferedEventRange(ProcessingState processingSt continue; } - ResultTypeT result = state.produceResult(); + ResultT result = state.produceResult(); if (result != null) { outputReceiver.get(mainOutputTupleTag).output(KV.of(processingState.getKey(), result)); processingState.resultProduced(); diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java index 92f9d92663b3..f1b935d6ef01 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java @@ -111,7 +111,7 @@ public int numberOfRanges() { public void merge(SequenceRangeAccumulator another) { if (this.initialSequence != null && another.initialSequence != null - && this.initialSequence.equals(another.initialSequence)) { + && ! this.initialSequence.equals(another.initialSequence)) { throw new IllegalStateException("Two accumulators contain different initial sequences: " + this.initialSequence + " and " + another.initialSequence); } diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java index c47f3ce8e18d..133773df66ca 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java @@ -9,7 +9,7 @@ public class SequenceRangeAccumulatorTest { - // Atomic ust in case tests are run in parallel + // Atomic just in case tests are run in parallel private final static AtomicLong currentTicker = new AtomicLong(); static Instant nextTimestamp() { @@ -128,7 +128,7 @@ public void testRemovingRangesBelowInitialSequenceDuringAccumulation() { new Event(7, nextTimestamp(), true), }; - doTestAccumulation(events, CompletedSequenceRange.of(7,7, eventTimestamp(events, 7)), 1); + doTestAccumulation(events, CompletedSequenceRange.of(7, 7, eventTimestamp(events, 7)), 1); } @Test @@ -151,7 +151,7 @@ public void testRemovingElementsBelowInitialSequenceDuringAccumulation() { }; Instant timestampOfTheLastEvent = events[events.length - 1].timestamp; - doTestAccumulation(events, CompletedSequenceRange.of(7,8, timestampOfTheLastEvent), 1); + doTestAccumulation(events, CompletedSequenceRange.of(7, 8, timestampOfTheLastEvent), 1); } private static void doTestAccumulation(Event[] events, CompletedSequenceRange expectedResult, @@ -233,7 +233,6 @@ public void testMergingWithoutAnyInitialEvents() { @Test public void testMergingAdjacentRanges() { - // TODO: Test the max timestamp being selected. Event[] set1 = new Event[]{ new Event(1, nextTimestamp(), true), new Event(2, nextTimestamp()), @@ -274,7 +273,66 @@ public void testPruningSequencesBelowInitial() { @Test public void testDuplicateHandling() { - // TODO: + Event[] set1 = new Event[]{ + new Event(1, nextTimestamp(), true), + new Event(2, nextTimestamp()), + new Event(3, nextTimestamp()), + new Event(5, nextTimestamp()), + }; + Event[] set2 = new Event[]{ + new Event(3, nextTimestamp()), + new Event(4, nextTimestamp()), + new Event(5, nextTimestamp()), + new Event(6, nextTimestamp()) + }; + + CompletedSequenceRange expectedResult = CompletedSequenceRange.of(1, 6, + eventTimestamp(set2, 6L)); + int expectedNumberOfRanges = 1; + + doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); + } + + @Test + public void testExceptionThrownIfThereAreDifferentInitialSequences() { + Event[] set1 = new Event[]{ + new Event(1, nextTimestamp(), true), + new Event(2, nextTimestamp()), + }; + Event[] set2 = new Event[]{ + new Event(3, nextTimestamp(), true), + new Event(4, nextTimestamp()), + new Event(5, nextTimestamp()), + new Event(6, nextTimestamp()) + }; + + try { + doTestMerging(set1, set2, CompletedSequenceRange.EMPTY, 0); + Assert.fail("Expected to throw an exception"); + } catch (IllegalStateException e) { + Assert.assertEquals("Exception message", + "Two accumulators contain different initial sequences: 1 and 3", e.getMessage()); + } + } + + + @Test + public void testSelectingHighestTimestampWhenMerging() { + Event[] set1 = new Event[]{ + new Event(1, nextTimestamp(), true), + new Event(2, Instant.ofEpochMilli(currentTicker.get() + 10000)), + }; + Event[] set2 = new Event[]{ + new Event(3, nextTimestamp()), + new Event(4, nextTimestamp()), + new Event(5, nextTimestamp()), + new Event(6, nextTimestamp()) + }; + + CompletedSequenceRange expectedResult = CompletedSequenceRange.of(1, 6, + eventTimestamp(set1, 2L)); + int expectedNumberOfRanges = 1; + doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); } private static void doTestMerging(Event[] set1, Event[] set2, From ca12f0c500b8cd8234f17176daf5fd23455c1c85 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Wed, 18 Sep 2024 19:50:18 -0700 Subject: [PATCH 16/33] Tests to verify that the the global sequence is correctly produced by the transform. --- .../ordered/CompletedSequenceRange.java | 6 +- .../ordered/GlobalSequencesProcessorDoFn.java | 2 +- .../ordered/OrderedEventProcessor.java | 155 +++++++++--------- .../ordered/OrderedEventProcessorResult.java | 36 +++- .../ordered/OrderedProcessingHandler.java | 40 ++--- .../ordered/PerKeyTickerGenerator.java | 6 +- .../sdk/extensions/ordered/ProcessorDoFn.java | 7 +- .../ordered/SequencePerKeyProcessorDoFn.java | 2 +- .../sdk/extensions/ordered/SequenceType.java | 6 - .../combiner/DefaultSequenceCombiner.java | 10 +- .../combiner/SequenceRangeAccumulator.java | 2 + ...deredEventProcessorGlobalSequenceTest.java | 62 ++++--- ...deredEventProcessorPerKeySequenceTest.java | 102 ++++++------ .../OrderedEventProcessorTestBase.java | 124 ++++++++++++-- .../StringBufferOrderedProcessingHandler.java | 18 ++ .../src/test/resources/logging.properties | 2 +- 16 files changed, 366 insertions(+), 214 deletions(-) delete mode 100644 sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequenceType.java diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/CompletedSequenceRange.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/CompletedSequenceRange.java index cacfd029f9e7..2e04d604231c 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/CompletedSequenceRange.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/CompletedSequenceRange.java @@ -4,6 +4,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.io.Serializable; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.DefaultCoder; @@ -19,6 +20,7 @@ @AutoValue public abstract class CompletedSequenceRange { +// private static final long serialVersionUID = 1L; public static final CompletedSequenceRange EMPTY = CompletedSequenceRange.of(Long.MIN_VALUE, Long.MIN_VALUE, Instant.ofEpochMilli( Long.MIN_VALUE)); @@ -27,10 +29,6 @@ public abstract class CompletedSequenceRange { public abstract long getEnd(); public abstract Instant getTimestamp(); - public boolean isEmpty() { - return this.equals(EMPTY); - } - public static CompletedSequenceRange of(long start, long end, Instant timestamp) { return new AutoValue_CompletedSequenceRange(start, end, timestamp); } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java index ac35ca21a48b..863fbca1cd57 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java @@ -93,7 +93,7 @@ class GlobalSequencesProcessorDoFn expand( throw new RuntimeException("Unable to get result coder", e); } - PCollectionTuple processingResult; - - switch (handler.getSequenceType()) { - case GLOBAL: - final PCollectionView latestContinuousSequence = - input - .apply("Convert to SequenceAndTimestamp", - ParDo.of(new ToTimestampedEventConverter<>())) - .apply("Global Sequence Tracker", - new GlobalSequenceTracker<>(handler.getGlobalSequenceCombiner())); - - PCollection>> tickers = input.apply("Create Tickers", - new PerKeyTickerGenerator<>(keyCoder, eventCoder, Duration.standardSeconds(5))); - - PCollection>> eventsAndTickers = - PCollectionList.of(input).and(tickers) - .apply("Combine Events and Tickers", Flatten.pCollections()) - .setCoder(tickers.getCoder()); - processingResult = - eventsAndTickers - .apply( - ParDo.of( - new GlobalSequencesProcessorDoFn<>( - handler.getEventExaminer(), - eventCoder, - stateCoder, - keyCoder, - mainOutput, - statusOutput, - handler.getStatusUpdateFrequency(), - unprocessedEventOutput, - handler.isProduceStatusUpdateOnEveryEvent(), - handler.getMaxOutputElementsPerBundle(), - latestContinuousSequence) - ) - .withOutputTags( - mainOutput, - TupleTagList.of(Arrays.asList(statusOutput, unprocessedEventOutput))) - .withSideInput(GLOBAL_SEQUENCE_TRACKER, latestContinuousSequence) - ); - break; - - case PER_KEY: - processingResult = - input.apply( - ParDo.of( - new SequencePerKeyProcessorDoFn( - handler.getEventExaminer(), - eventCoder, - stateCoder, - keyCoder, - mainOutput, - statusOutput, - handler.getStatusUpdateFrequency(), - unprocessedEventOutput, - handler.isProduceStatusUpdateOnEveryEvent(), - handler.getMaxOutputElementsPerBundle())) - .withOutputTags( - mainOutput, - TupleTagList.of(Arrays.asList(statusOutput, unprocessedEventOutput)))); - break; - - default: - throw new IllegalStateException("Unprocessed sequence type: " + handler.getSequenceType()); - } - KvCoder mainOutputCoder = KvCoder.of(keyCoder, resultCoder); KvCoder processingStatusCoder = KvCoder.of(keyCoder, getOrderedProcessingStatusCoder(pipeline)); KvCoder>> unprocessedEventsCoder = KvCoder.of( keyCoder, KvCoder.of(VarLongCoder.of(), new UnprocessedEventCoder<>(eventCoder))); - return new OrderedEventProcessorResult<>( - pipeline, - processingResult.get(mainOutput).setCoder(mainOutputCoder), - mainOutput, - processingResult.get(statusOutput).setCoder(processingStatusCoder), - statusOutput, - processingResult.get(unprocessedEventOutput).setCoder(unprocessedEventsCoder), - unprocessedEventOutput); + + PCollectionTuple processingResult; + + if (handler instanceof OrderedProcessingGlobalSequenceHandler) { + OrderedProcessingGlobalSequenceHandler globalSequenceHandler = (OrderedProcessingGlobalSequenceHandler) handler; + final PCollectionView latestContinuousSequence = + input + .apply("Convert to SequenceAndTimestamp", + ParDo.of(new ToTimestampedEventConverter<>())) + .apply("Global Sequence Tracker", + new GlobalSequenceTracker<>(globalSequenceHandler.getGlobalSequenceCombiner())); + + PCollection>> tickers = input.apply("Create Tickers", + new PerKeyTickerGenerator<>(keyCoder, eventCoder, + globalSequenceHandler.getFrequencyOfCheckingForNewGlobalSequence())); + + PCollection>> eventsAndTickers = + PCollectionList.of(input).and(tickers) + .apply("Combine Events and Tickers", Flatten.pCollections()) + .setCoder(tickers.getCoder()); + processingResult = + eventsAndTickers + .apply( + ParDo.of( + new GlobalSequencesProcessorDoFn<>( + handler.getEventExaminer(), + eventCoder, + stateCoder, + keyCoder, + mainOutput, + statusOutput, + handler.getStatusUpdateFrequency(), + unprocessedEventOutput, + handler.isProduceStatusUpdateOnEveryEvent(), + handler.getMaxOutputElementsPerBundle(), + latestContinuousSequence) + ) + .withOutputTags( + mainOutput, + TupleTagList.of(Arrays.asList(statusOutput, unprocessedEventOutput))) + .withSideInput(GLOBAL_SEQUENCE_TRACKER, latestContinuousSequence) + ); + return new OrderedEventProcessorResult<>( + pipeline, + processingResult.get(mainOutput).setCoder(mainOutputCoder), + mainOutput, + processingResult.get(statusOutput).setCoder(processingStatusCoder), + statusOutput, + processingResult.get(unprocessedEventOutput).setCoder(unprocessedEventsCoder), + unprocessedEventOutput, + latestContinuousSequence); + } else { + // Per key sequence handler + processingResult = + input.apply( + ParDo.of( + new SequencePerKeyProcessorDoFn<>( + handler.getEventExaminer(), + eventCoder, + stateCoder, + keyCoder, + mainOutput, + statusOutput, + handler.getStatusUpdateFrequency(), + unprocessedEventOutput, + handler.isProduceStatusUpdateOnEveryEvent(), + handler.getMaxOutputElementsPerBundle())) + .withOutputTags( + mainOutput, + TupleTagList.of(Arrays.asList(statusOutput, unprocessedEventOutput)))); + return new OrderedEventProcessorResult<>( + pipeline, + processingResult.get(mainOutput).setCoder(mainOutputCoder), + mainOutput, + processingResult.get(statusOutput).setCoder(processingStatusCoder), + statusOutput, + processingResult.get(unprocessedEventOutput).setCoder(unprocessedEventsCoder), + unprocessedEventOutput); + + } } private static Coder getOrderedProcessingStatusCoder(Pipeline pipeline) { diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorResult.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorResult.java index f61df6254b25..c55dc1330d0f 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorResult.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorResult.java @@ -18,10 +18,12 @@ package org.apache.beam.sdk.extensions.ordered; import java.util.Map; +import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; @@ -29,8 +31,10 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; /** - * The result of the ordered processing. Two PCollections are returned: + * The result of the ordered processing. Three PCollections are returned: *

  • output - the key/value of the mutated states + *
  • unprocessedEvents - the key/value of the events that failed to be processed and the failure + * reason *
  • processingStatuses - the key/value of the status of processing for a particular key * * @param @@ -48,6 +52,8 @@ public class OrderedEventProcessorResult implements POutp unprocessedEventPCollection; private final TupleTag>>> unprocessedEventTupleTag; + private final @Nullable PCollectionView latestCompletedSequenceRange; + OrderedEventProcessorResult( Pipeline pipeline, PCollection> outputPCollection, @@ -57,6 +63,20 @@ public class OrderedEventProcessorResult implements POutp PCollection>>> unprocessedEventPCollection, TupleTag>>> unprocessedEventTupleTag) { + this(pipeline, outputPCollection, outputPCollectionTupleTag, eventProcessingStatusPCollection, + eventProcessingStatusTupleTag, unprocessedEventPCollection, unprocessedEventTupleTag, null); + } + + OrderedEventProcessorResult( + Pipeline pipeline, + PCollection> outputPCollection, + TupleTag> outputPCollectionTupleTag, + PCollection> eventProcessingStatusPCollection, + TupleTag> eventProcessingStatusTupleTag, + PCollection>>> unprocessedEventPCollection, + TupleTag>>> unprocessedEventTupleTag, + @Nullable PCollectionView latestCompletedSequenceRange) { + this.pipeline = pipeline; this.outputPCollection = outputPCollection; this.outputPCollectionTupleTag = outputPCollectionTupleTag; @@ -64,6 +84,7 @@ public class OrderedEventProcessorResult implements POutp this.eventProcessingStatusTupleTag = eventProcessingStatusTupleTag; this.unprocessedEventPCollection = unprocessedEventPCollection; this.unprocessedEventTupleTag = unprocessedEventTupleTag; + this.latestCompletedSequenceRange = latestCompletedSequenceRange; } private final Pipeline pipeline; @@ -86,17 +107,20 @@ public Map, PValue> expand() { @Override public void finishSpecifyingOutput( - String transformName, PInput input, PTransform transform) {} + String transformName, PInput input, PTransform transform) { + } /** * @return processing status for a particular key. The elements will have the timestamp of the - * instant the status was emitted. + * instant the status was emitted. */ public PCollection> processingStatuses() { return eventProcessingStatusPCollection; } - /** @return processed states keyed by the original key */ + /** + * @return processed states keyed by the original key + */ public PCollection> output() { return outputPCollection; } @@ -104,4 +128,8 @@ public PCollection> output() { public PCollection>>> unprocessedEvents() { return unprocessedEventPCollection; } + + public @Nullable PCollectionView latestCompletedSequenceRange() { + return latestCompletedSequenceRange; + } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java index be6b5ad2b8ac..786b3bd87fb3 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java @@ -27,10 +27,8 @@ import org.apache.beam.sdk.transforms.Combine.GloballyAsSingletonView; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TimestampedValue; -import org.checkerframework.checker.initialization.qual.Initialized; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; -import org.checkerframework.checker.nullness.qual.UnknownKeyFor; import org.joda.time.Duration; /** @@ -59,9 +57,6 @@ public abstract class OrderedProcessingHandler< Duration.standardSeconds(DEFAULT_STATUS_UPDATE_FREQUENCY_SECONDS); private boolean produceStatusUpdateOnEveryEvent = DEFAULT_PRODUCE_STATUS_UPDATE_ON_EVERY_EVENT; - private SequenceType sequenceType = SequenceType.PER_KEY; - private @Nullable GloballyAsSingletonView>>, CompletedSequenceRange> globalSequenceCombiner; - /** * Provide concrete classes which will be used by the ordered processing transform. * @@ -79,7 +74,6 @@ public OrderedProcessingHandler( this.keyTClass = keyTClass; this.stateTClass = stateTClass; this.resultTClass = resultTClass; - this.globalSequenceCombiner = null; } /** @@ -230,29 +224,25 @@ public void setMaxOutputElementsPerBundle(int maxOutputElementsPerBundle) { this.maxOutputElementsPerBundle = maxOutputElementsPerBundle; } - public SequenceType getSequenceType() { - return sequenceType; - } - - public void setSequenceType(SequenceType sequenceType) { - this.sequenceType = sequenceType; - } + public abstract static class OrderedProcessingGlobalSequenceHandler< + EventT, KeyT, StateT extends MutableState, ResultT> extends + OrderedProcessingHandler { - public GloballyAsSingletonView>>, CompletedSequenceRange> getGlobalSequenceCombiner() { - if (getSequenceType() != SequenceType.GLOBAL) { - throw new IllegalStateException( - "Global Sequence Combiner is only useful when the sequence is global. Current sequence type: " - + getSequenceType()); + public OrderedProcessingGlobalSequenceHandler( + Class eventTClass, + Class keyTClass, + Class stateTClass, + Class resultTClass) { + super(eventTClass, keyTClass, stateTClass, resultTClass); } - if (globalSequenceCombiner == null) { - globalSequenceCombiner = Combine.globally( + + public GloballyAsSingletonView>>, CompletedSequenceRange> getGlobalSequenceCombiner() { + return Combine.globally( new DefaultSequenceCombiner(getEventExaminer())).asSingletonView(); } - return globalSequenceCombiner; - } - public void setGlobalSequenceCombiner( - GloballyAsSingletonView>>, CompletedSequenceRange> globalSequenceCombiner) { - this.globalSequenceCombiner = globalSequenceCombiner; + public Duration getFrequencyOfCheckingForNewGlobalSequence() { + return Duration.standardSeconds(1); + } } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/PerKeyTickerGenerator.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/PerKeyTickerGenerator.java index 43f278f30dd3..24bd4465e2a3 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/PerKeyTickerGenerator.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/PerKeyTickerGenerator.java @@ -75,11 +75,9 @@ static class PerKeyTickerGeneratorDoFn @ProcessElement public void process( @Element KV> element, + @AlwaysFetched @StateId(STATE) ValueState state, @TimerId(TIMER) Timer tickerTimer) { - // Keys are usually simple types. The difference in cost of reading those types from - // state storage of those types comparing to the cost of reading a Boolean should be - // negligible. @Nullable EventKeyT keyValue = state.read(); if (keyValue != null) { return; @@ -102,9 +100,9 @@ public void onTimer( return; } + // null value will be an indicator to the main transform that the element is a ticker outputReceiver.output(KV.of(key, KV.of(0L, null))); tickerTimer.offset(tickerFrequency).setRelative(); } - } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java index a596bd00fb1f..710a5353a15f 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java @@ -81,7 +81,7 @@ public void onBundleFinish() { /** * @return true if each event needs to be examined. */ - abstract boolean checkForInitialEvent(); + abstract boolean checkForFirstOrLastEvent(); /** * Process the just received event. @@ -122,8 +122,9 @@ public void onBundleFinish() { } StateT state; - boolean thisIsTheLastEvent = eventExaminer.isLastEvent(currentSequence, currentEvent); - if (checkForInitialEvent() && eventExaminer.isInitialEvent(currentSequence, currentEvent)) { + boolean thisIsTheLastEvent = checkForFirstOrLastEvent() + && eventExaminer.isLastEvent(currentSequence, currentEvent); + if (checkForFirstOrLastEvent() && eventExaminer.isInitialEvent(currentSequence, currentEvent)) { // First event of the key/window // What if it's a duplicate event - it will reset everything. Shall we drop/DLQ anything // that's before the processingState.lastOutputSequence? diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java index f47f667718f5..4d073dc204ba 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java @@ -83,7 +83,7 @@ class SequencePerKeyProcessorDoFn> extends CombineFn>>, SequenceRangeAccumulator, CompletedSequenceRange> { + private static final Logger LOG = LoggerFactory.getLogger(DefaultSequenceCombiner.class); + public static final BiFunction<@NonNull Instant, @Nullable Instant, @Nullable Instant> OLDEST_TIMESTAMP_SELECTOR = (instant1, instant2) -> { if (instant2 == null) { return instant1; @@ -64,7 +68,11 @@ public SequenceRangeAccumulator mergeAccumulators( @Override public CompletedSequenceRange extractOutput(SequenceRangeAccumulator accum) { - return accum.largestContinuousRange(); + CompletedSequenceRange result = accum.largestContinuousRange(); + if(LOG.isTraceEnabled()) { + LOG.trace("Returning completed sequence range: " + result); + } + return result; } @Override diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java index f1b935d6ef01..63c8dcf0be96 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java @@ -19,6 +19,8 @@ import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.UnknownKeyFor; import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class SequenceRangeAccumulator { diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java index 4dc9e077706c..b575cd55cd2a 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java @@ -6,6 +6,7 @@ import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.extensions.ordered.StringBufferOrderedProcessingHandler.StringBufferOrderedProcessingWithGlobalSequenceHandler; import org.apache.beam.sdk.extensions.ordered.UnprocessedEvent.Reason; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestStream; @@ -22,6 +23,8 @@ public class OrderedEventProcessorGlobalSequenceTest extends OrderedEventProcessorTestBase { + public static final boolean GLOBAL_SEQUENCE = true; + @org.junit.Test public void testPerfectOrderingProcessing() throws CannotProvideCoderException { Event[] events = { @@ -46,7 +49,8 @@ public void testPerfectOrderingProcessing() throws CannotProvideCoderException { expectedOutput, EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT); + LARGE_MAX_RESULTS_PER_OUTPUT, + CompletedSequenceRange.of(0,5, new Instant())); } @Test @@ -79,7 +83,8 @@ public void testOutOfSequenceProcessing() throws CannotProvideCoderException { expectedOutput, EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT); + LARGE_MAX_RESULTS_PER_OUTPUT, + CompletedSequenceRange.of(0,8, new Instant())); } @Test @@ -118,7 +123,8 @@ public void testHandlingOfDuplicateSequences() throws CannotProvideCoderExceptio duplicates, EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT); + LARGE_MAX_RESULTS_PER_OUTPUT, + CompletedSequenceRange.of(0,3, new Instant())); } @Test @@ -151,7 +157,8 @@ public void testTreatingSequencesBelowInitialAsDuplicates() throws CannotProvide duplicates, EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT); + LARGE_MAX_RESULTS_PER_OUTPUT, + CompletedSequenceRange.of(0,3, new Instant())); } @Test @@ -185,7 +192,11 @@ public void testHandlingOfCheckedExceptions() throws CannotProvideCoderException failedEvents, EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT); + LARGE_MAX_RESULTS_PER_OUTPUT, + // Sequence matcher doesn't know if the element is valid or not. + // That's why the elements that are get rejected in the processor still count when + // calculating the global sequence + CompletedSequenceRange.of(0,3, new Instant())); } @Test @@ -211,7 +222,8 @@ public void testProcessingWithEveryOtherResultEmission() throws CannotProvideCod expectedOutput, EMISSION_FREQUENCY_ON_EVERY_OTHER_EVENT, INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT); + LARGE_MAX_RESULTS_PER_OUTPUT, + CompletedSequenceRange.of(0,5, new Instant())); } @Test @@ -245,7 +257,8 @@ public void testLargeBufferedOutputInTimer() throws CannotProvideCoderException expectedOutput, EMISSION_FREQUENCY_ON_EVERY_ELEMENT, 1L /* This dataset assumes 1 as the starting sequence */, - maxResultsPerOutput); + maxResultsPerOutput, + CompletedSequenceRange.of(1, sequences.length, new Instant())); } @Test @@ -275,20 +288,23 @@ public void testSequenceGapProcessingInBufferedOutput() throws CannotProvideCode expectedOutput, EMISSION_FREQUENCY_ON_EVERY_ELEMENT, 1L /* This dataset assumes 1 as the starting sequence */, - maxResultsPerOutput); + maxResultsPerOutput, + CompletedSequenceRange.of(1,10, new Instant())); } @Test public void testHandlingOfMaxSequenceNumber() throws CannotProvideCoderException { Event[] events = { - Event.create(0, "id-1", "a"), Event.create(1, "id-1", "b"), - Event.create(Long.MAX_VALUE, "id-1", "c") + Event.create(0, "id-1", "a"), + Event.create(Long.MAX_VALUE, "id-1", "d"), + Event.create(2, "id-1", "c") }; Collection> expectedOutput = new ArrayList<>(); expectedOutput.add(KV.of("id-1", "a")); expectedOutput.add(KV.of("id-1", "ab")); + expectedOutput.add(KV.of("id-1", "abc")); Collection>>> unprocessedEvents = new ArrayList<>(); @@ -297,7 +313,7 @@ public void testHandlingOfMaxSequenceNumber() throws CannotProvideCoderException "id-1", KV.of( Long.MAX_VALUE, - UnprocessedEvent.create("c", Reason.sequence_id_outside_valid_range)))); + UnprocessedEvent.create("d", Reason.sequence_id_outside_valid_range)))); testGlobalSequenceProcessing( events, @@ -305,7 +321,8 @@ public void testHandlingOfMaxSequenceNumber() throws CannotProvideCoderException unprocessedEvents, EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT); + LARGE_MAX_RESULTS_PER_OUTPUT, + CompletedSequenceRange.of(0, 2, Instant.now())); } @Test @@ -327,7 +344,8 @@ public void testProcessingOfTheLastInput() throws CannotProvideCoderException { expectedOutput, EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, - LARGE_MAX_RESULTS_PER_OUTPUT); + LARGE_MAX_RESULTS_PER_OUTPUT, + CompletedSequenceRange.of(0,2, new Instant())); } @@ -336,7 +354,8 @@ private void testGlobalSequenceProcessing( Collection> expectedOutput, int emissionFrequency, long initialSequence, - int maxResultsPerOutput) + int maxResultsPerOutput, + CompletedSequenceRange expectedLastCompleteRange) throws CannotProvideCoderException { testGlobalSequenceProcessing( events, @@ -344,7 +363,7 @@ private void testGlobalSequenceProcessing( NO_EXPECTED_DLQ_EVENTS, emissionFrequency, initialSequence, - maxResultsPerOutput); + maxResultsPerOutput, expectedLastCompleteRange); } private void testGlobalSequenceProcessing( @@ -353,7 +372,8 @@ private void testGlobalSequenceProcessing( Collection>>> expectedUnprocessedEvents, int emissionFrequency, long initialSequence, - int maxResultsPerOutput) + int maxResultsPerOutput, + CompletedSequenceRange expectedLastCompleteRange) throws CannotProvideCoderException { // Test a streaming pipeline doTest( @@ -366,7 +386,7 @@ private void testGlobalSequenceProcessing( maxResultsPerOutput, false /* produceStatusOnEveryEvent */, STREAMING, - SequenceType.GLOBAL); + GLOBAL_SEQUENCE, expectedLastCompleteRange); if (true) { // TODO: Test batch processing @@ -383,7 +403,8 @@ private void testGlobalSequenceProcessing( maxResultsPerOutput, false /* produceStatusOnEveryEvent */, BATCH, - SequenceType.GLOBAL); + GLOBAL_SEQUENCE, + expectedLastCompleteRange); } @@ -423,13 +444,12 @@ public void testWindowedProcessing() throws CannotProvideCoderException { input = input.apply("Window input", Window.into(FixedWindows.of(Duration.standardSeconds(5)))); - StringBufferOrderedProcessingHandler handler = - new StringBufferOrderedProcessingHandler( + StringBufferOrderedProcessingWithGlobalSequenceHandler handler = + new StringBufferOrderedProcessingWithGlobalSequenceHandler( EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0); handler.setMaxOutputElementsPerBundle(LARGE_MAX_RESULTS_PER_OUTPUT); handler.setStatusUpdateFrequency(null); handler.setProduceStatusUpdateOnEveryEvent(false); - handler.setSequenceType(SequenceType.GLOBAL); OrderedEventProcessor orderedEventProcessor = OrderedEventProcessor.create(handler); diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java index 3cebf2019dac..a3e98275636c 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java @@ -25,12 +25,12 @@ public class OrderedEventProcessorPerKeySequenceTest extends OrderedEventProcess @Test public void testPerfectOrderingProcessing() throws CannotProvideCoderException { Event[] events = { - Event.create(0, "id-1", "a"), - Event.create(1, "id-1", "b"), - Event.create(2, "id-1", "c"), - Event.create(3, "id-1", "d"), - Event.create(0, "id-2", "a"), - Event.create(1, "id-2", "b") + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + Event.create(2, "id-1", "c"), + Event.create(3, "id-1", "d"), + Event.create(0, "id-2", "a"), + Event.create(1, "id-2", "b") }; Collection> expectedStatuses = new ArrayList<>(); @@ -80,15 +80,15 @@ public void testPerfectOrderingProcessing() throws CannotProvideCoderException { @Test public void testOutOfSequenceProcessing() throws CannotProvideCoderException { Event[] events = { - Event.create(2, "id-1", "c"), - Event.create(1, "id-1", "b"), - Event.create(0, "id-1", "a"), - Event.create(3, "id-1", "d"), - Event.create(1, "id-2", "b"), - Event.create(2, "id-2", "c"), - Event.create(4, "id-2", "e"), - Event.create(0, "id-2", "a"), - Event.create(3, "id-2", "d") + Event.create(2, "id-1", "c"), + Event.create(1, "id-1", "b"), + Event.create(0, "id-1", "a"), + Event.create(3, "id-1", "d"), + Event.create(1, "id-2", "b"), + Event.create(2, "id-2", "c"), + Event.create(4, "id-2", "e"), + Event.create(0, "id-2", "a"), + Event.create(3, "id-2", "d") }; Collection> expectedStatuses = new ArrayList<>(); @@ -141,12 +141,12 @@ public void testOutOfSequenceProcessing() throws CannotProvideCoderException { @Test public void testUnfinishedProcessing() throws CannotProvideCoderException { Event[] events = { - Event.create(2, "id-1", "c"), - // Excluded Event.create(1, "id-1", "b"), - Event.create(0, "id-1", "a"), - Event.create(3, "id-1", "d"), - Event.create(0, "id-2", "a"), - Event.create(1, "id-2", "b"), + Event.create(2, "id-1", "c"), + // Excluded Event.create(1, "id-1", "b"), + Event.create(0, "id-1", "a"), + Event.create(3, "id-1", "d"), + Event.create(0, "id-2", "a"), + Event.create(1, "id-2", "b"), }; Collection> expectedStatuses = new ArrayList<>(); @@ -166,17 +166,17 @@ public void testUnfinishedProcessing() throws CannotProvideCoderException { @Test public void testHandlingOfDuplicateSequences() throws CannotProvideCoderException { Event[] events = { - Event.create(3, "id-1", "d"), - Event.create(2, "id-1", "c"), - // Duplicates to be buffered - Event.create(3, "id-1", "d"), - Event.create(3, "id-1", "d"), - Event.create(0, "id-1", "a"), - Event.create(1, "id-1", "b"), - - // Duplicates after the events are processed - Event.create(1, "id-1", "b"), - Event.create(3, "id-1", "d"), + Event.create(3, "id-1", "d"), + Event.create(2, "id-1", "c"), + // Duplicates to be buffered + Event.create(3, "id-1", "d"), + Event.create(3, "id-1", "d"), + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + + // Duplicates after the events are processed + Event.create(1, "id-1", "b"), + Event.create(3, "id-1", "d"), }; int resultCount = 4; int duplicateCount = 4; @@ -214,10 +214,10 @@ public void testHandlingOfDuplicateSequences() throws CannotProvideCoderExceptio @Test public void testHandlingOfCheckedExceptions() throws CannotProvideCoderException { Event[] events = { - Event.create(0, "id-1", "a"), - Event.create(1, "id-1", "b"), - Event.create(2, "id-1", StringBuilderState.BAD_VALUE), - Event.create(3, "id-1", "c"), + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + Event.create(2, "id-1", StringBuilderState.BAD_VALUE), + Event.create(3, "id-1", "c"), }; Collection> expectedStatuses = new ArrayList<>(); @@ -250,12 +250,12 @@ public void testHandlingOfCheckedExceptions() throws CannotProvideCoderException @Test public void testProcessingWithEveryOtherResultEmission() throws CannotProvideCoderException { Event[] events = { - Event.create(2, "id-1", "c"), - Event.create(1, "id-1", "b"), - Event.create(0, "id-1", "a"), - Event.create(3, "id-1", "d"), - Event.create(0, "id-2", "a"), - Event.create(1, "id-2", "b"), + Event.create(2, "id-1", "c"), + Event.create(1, "id-1", "b"), + Event.create(0, "id-1", "a"), + Event.create(3, "id-1", "d"), + Event.create(0, "id-2", "a"), + Event.create(1, "id-2", "b"), }; Collection> expectedStatuses = new ArrayList<>(); @@ -367,7 +367,7 @@ public void testLargeBufferedOutputInTimer() throws CannotProvideCoderException public void testSequenceGapProcessingInBufferedOutput() throws CannotProvideCoderException { int maxResultsPerOutput = 3; - long[] sequences = new long[] {2, 3, 7, 8, 9, 10, 1, 4, 5, 6}; + long[] sequences = new long[]{2, 3, 7, 8, 9, 10, 1, 4, 5, 6}; List events = new ArrayList<>(sequences.length); List> expectedOutput = new ArrayList<>(sequences.length); @@ -461,9 +461,9 @@ public void testSequenceGapProcessingInBufferedOutput() throws CannotProvideCode @Test public void testHandlingOfMaxSequenceNumber() throws CannotProvideCoderException { Event[] events = { - Event.create(0, "id-1", "a"), - Event.create(1, "id-1", "b"), - Event.create(Long.MAX_VALUE, "id-1", "c") + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + Event.create(Long.MAX_VALUE, "id-1", "c") }; Collection> expectedStatuses = new ArrayList<>(); @@ -497,9 +497,9 @@ public void testHandlingOfMaxSequenceNumber() throws CannotProvideCoderException @Test public void testProcessingOfTheLastInput() throws CannotProvideCoderException { Event[] events = { - Event.create(0, "id-1", "a"), - Event.create(1, "id-1", "b"), - Event.create(2, "id-1", StringEventExaminer.LAST_INPUT) + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + Event.create(2, "id-1", StringEventExaminer.LAST_INPUT) }; Collection> expectedStatuses = new ArrayList<>(); @@ -565,7 +565,7 @@ protected void testPerKeySequenceProcessing( maxResultsPerOutput, produceStatusOnEveryEvent, STREAMING, - SequenceType.PER_KEY); + false, CompletedSequenceRange.EMPTY); // Test a batch pipeline doTest( @@ -578,7 +578,7 @@ protected void testPerKeySequenceProcessing( maxResultsPerOutput, produceStatusOnEveryEvent, BATCH, - SequenceType.PER_KEY); + false, CompletedSequenceRange.EMPTY); } @Test diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTestBase.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTestBase.java index 2043b2cd0b59..21ce703e890b 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTestBase.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTestBase.java @@ -17,13 +17,18 @@ */ package org.apache.beam.sdk.extensions.ordered; +import static org.hamcrest.MatcherAssert.assertThat; + import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.Iterator; import java.util.Set; import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.extensions.ordered.StringBufferOrderedProcessingHandler.StringBufferOrderedProcessingWithGlobalSequenceHandler; import org.apache.beam.sdk.extensions.ordered.UnprocessedEvent.Reason; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.SerializableMatcher; @@ -32,14 +37,17 @@ import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; 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.transforms.Reshuffle; +import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Repeatedly; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; import org.checkerframework.checker.initialization.qual.Initialized; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.UnknownKeyFor; @@ -48,15 +56,13 @@ import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Rule; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; /** * Ordered Processing tests use the same testing scenario. Events are sent in or out of sequence. * Each event is a string for a particular key. The output is a concatenation of all strings. */ -@RunWith(JUnit4.class) public class OrderedEventProcessorTestBase { + public static final boolean LAST_EVENT_RECEIVED = true; public static final int EMISSION_FREQUENCY_ON_EVERY_ELEMENT = 1; public static final int INITIAL_SEQUENCE_OF_0 = 0; @@ -96,16 +102,6 @@ public void map( /** * The majority of the tests use this method. Testing is done in the global window. * - * @param events - * @param expectedStatuses - * @param expectedOutput - * @param expectedUnprocessedEvents - * @param emissionFrequency - * @param initialSequence - * @param maxResultsPerOutput - * @param produceStatusOnEveryEvent - * @param streaming - * @param sequenceType * @throws @UnknownKeyFor @NonNull @Initialized CannotProvideCoderException */ protected void doTest( @@ -118,7 +114,8 @@ protected void doTest( int maxResultsPerOutput, boolean produceStatusOnEveryEvent, boolean streaming, - SequenceType sequenceType) + boolean isGlobalSequence, + @Nullable CompletedSequenceRange expectedLastCompletedSequence) throws @UnknownKeyFor @NonNull @Initialized CannotProvideCoderException { Pipeline pipeline = streaming ? streamingPipeline : batchPipeline; @@ -130,8 +127,11 @@ protected void doTest( PCollection>> input = rawInput.apply("To KV", ParDo.of(new MapEventsToKV())); - StringBufferOrderedProcessingHandler handler = - new StringBufferOrderedProcessingHandler(emissionFrequency, initialSequence); + OrderedProcessingHandler handler = + isGlobalSequence ? + new StringBufferOrderedProcessingWithGlobalSequenceHandler(emissionFrequency, + initialSequence) : + new StringBufferOrderedProcessingHandler(emissionFrequency, initialSequence); handler.setMaxOutputElementsPerBundle(maxResultsPerOutput); if (produceStatusOnEveryEvent) { handler.setProduceStatusUpdateOnEveryEvent(true); @@ -142,8 +142,6 @@ protected void doTest( streaming ? Duration.standardMinutes(5) : Duration.standardSeconds(1)); } - handler.setSequenceType(sequenceType); - OrderedEventProcessor orderedEventProcessor = OrderedEventProcessor.create(handler); @@ -186,9 +184,56 @@ protected void doTest( PAssert.that("Unprocessed events match", processingResult.unprocessedEvents()) .containsInAnyOrder(expectedUnprocessedEvents); } + + if (expectedLastCompletedSequence != null + && processingResult.latestCompletedSequenceRange() != null) { + PCollection globalSequences = rawInput + .apply("Publish Global Sequences", + new GlobalSequenceRangePublisher(processingResult.latestCompletedSequenceRange(), + handler.getKeyCoder(pipeline, input.getCoder()), + handler.getEventCoder(pipeline, input.getCoder()))); + PAssert.that("CompletedSequenceRange verification", globalSequences).satisfies( + new LastExpectedGlobalSequenceRangeMatcher(expectedLastCompletedSequence) + ); + } pipeline.run(); } + static class LastExpectedGlobalSequenceRangeMatcher implements + SerializableFunction, Void> { + + private final long expectedStart; + private final long expectedEnd; + + LastExpectedGlobalSequenceRangeMatcher(CompletedSequenceRange expected) { + this.expectedStart = expected.getStart(); + this.expectedEnd = expected.getEnd(); + } + + @Override + public Void apply(Iterable input) { + StringBuilder listOfRanges = new StringBuilder("["); + Iterator iterator = input.iterator(); + CompletedSequenceRange lastRange = null; + while (iterator.hasNext()) { + lastRange = iterator.next(); + + if (listOfRanges.length() > 1) { + listOfRanges.append(", "); + } + listOfRanges.append(lastRange); + } + listOfRanges.append(']'); + boolean foundExpectedRange = lastRange != null && + lastRange.getStart() == expectedStart && lastRange.getEnd() == expectedEnd; + + assertThat( + "Expected range not found: [" + expectedStart + '-' + expectedEnd + "], received ranges: " + + listOfRanges, foundExpectedRange); + return null; + } + } + private @UnknownKeyFor @NonNull @Initialized PCollection createBatchPCollection( Pipeline pipeline, Event[] events) { return pipeline @@ -264,4 +309,47 @@ static String normalizeExplanation(String value) { return value; } } + + static class GlobalSequenceRangePublisher extends + PTransform, PCollection> { + + private final PCollectionView lastCompletedSequenceRangeView; + private final Coder keyCoder; + private final Coder eventCoder; + + public GlobalSequenceRangePublisher( + PCollectionView latestCompletedSequenceRange, + Coder keyCoder, Coder eventCoder) { + this.lastCompletedSequenceRangeView = latestCompletedSequenceRange; + this.keyCoder = keyCoder; + this.eventCoder = eventCoder; + } + + @Override + public PCollection expand(PCollection input) { + return + input + // In production pipelines the global sequence will typically be obtained + // by using GenerateSequence. But GenerateSequence doesn't work well with TestStream, + // That's why we use the input events here. +// .apply("Create Ticker", +// GenerateSequence.from(0).to(2).withRate(1, Duration.standardSeconds(5))) + .apply("To KV", ParDo.of(new MapEventsToKV())) + .apply("Create Tickers", + new PerKeyTickerGenerator<>(keyCoder, eventCoder, + Duration.standardSeconds(1))) + .apply("Emit SideInput", ParDo.of(new SideInputEmitter()) + .withSideInput("lastCompletedSequence", lastCompletedSequenceRangeView)); + } + } + + static class SideInputEmitter extends DoFn>, CompletedSequenceRange> { + + @ProcessElement + public void produceCompletedRange( + @SideInput("lastCompletedSequence") CompletedSequenceRange sideInput, + OutputReceiver outputReceiver) { + outputReceiver.output(sideInput); + } + } } diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/StringBufferOrderedProcessingHandler.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/StringBufferOrderedProcessingHandler.java index 72f3a3cf21b6..aed3e78b9de1 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/StringBufferOrderedProcessingHandler.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/StringBufferOrderedProcessingHandler.java @@ -27,6 +27,24 @@ public class StringBufferOrderedProcessingHandler extends OrderedProcessingHandler { + public static class StringBufferOrderedProcessingWithGlobalSequenceHandler + extends OrderedProcessingGlobalSequenceHandler { + + private final EventExaminer eventExaminer; + + public StringBufferOrderedProcessingWithGlobalSequenceHandler(int emissionFrequency, + long initialSequence) { + super(String.class, String.class, StringBuilderState.class, String.class); + this.eventExaminer = new StringEventExaminer(initialSequence, emissionFrequency); + } + + @Override + @NonNull + public EventExaminer getEventExaminer() { + return eventExaminer; + } + } + private final EventExaminer eventExaminer; public StringBufferOrderedProcessingHandler(int emissionFrequency, long initialSequence) { diff --git a/sdks/java/extensions/ordered/src/test/resources/logging.properties b/sdks/java/extensions/ordered/src/test/resources/logging.properties index c324d864a083..7e6f820baafd 100644 --- a/sdks/java/extensions/ordered/src/test/resources/logging.properties +++ b/sdks/java/extensions/ordered/src/test/resources/logging.properties @@ -9,7 +9,7 @@ java.util.logging.SimpleFormatter.format = [%1$tc] %4$s: %2$s - %5$s %6$s%n # OFF, SEVERE, WARNING, INFO, CONFIG, FINE, FINER, FINEST, ALL # root logger -.level = INFO +.level = TRACE # child logger org.example.level = ALL \ No newline at end of file From a7ff45d427c185ac5196034c26ec54d68cea4566 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Mon, 23 Sep 2024 09:53:45 -0700 Subject: [PATCH 17/33] Added batch processing verification to the global sequence processing. --- sdks/java/extensions/ordered/build.gradle | 4 + ...ange.java => ContiguousSequenceRange.java} | 38 +++++--- .../sdk/extensions/ordered/EventExaminer.java | 9 +- .../ordered/GlobalSequenceTracker.java | 62 ++++++++----- .../ordered/GlobalSequencesProcessorDoFn.java | 47 ++++------ .../ordered/OrderedEventProcessor.java | 36 ++++--- .../ordered/OrderedEventProcessorResult.java | 6 +- .../ordered/OrderedProcessingHandler.java | 6 +- .../extensions/ordered/ProcessingState.java | 16 ++-- .../sdk/extensions/ordered/ProcessorDoFn.java | 6 +- .../ordered/SequencePerKeyProcessorDoFn.java | 2 +- .../combiner/DefaultSequenceCombiner.java | 24 +++-- .../combiner/SequenceRangeAccumulator.java | 26 ++++-- ...deredEventProcessorGlobalSequenceTest.java | 90 ++++++++++-------- ...deredEventProcessorPerKeySequenceTest.java | 4 +- .../OrderedEventProcessorTestBase.java | 93 ++++++++++++------- .../SequenceRangeAccumulatorCoderTest.java | 2 +- .../SequenceRangeAccumulatorTest.java | 42 ++++----- .../src/test/resources/logging.properties | 2 +- 19 files changed, 310 insertions(+), 205 deletions(-) rename sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/{CompletedSequenceRange.java => ContiguousSequenceRange.java} (65%) diff --git a/sdks/java/extensions/ordered/build.gradle b/sdks/java/extensions/ordered/build.gradle index 2338b4f72b47..8bee1901bd3a 100644 --- a/sdks/java/extensions/ordered/build.gradle +++ b/sdks/java/extensions/ordered/build.gradle @@ -31,5 +31,9 @@ dependencies { testImplementation library.java.slf4j_jdk14 testImplementation project(path: ':sdks:java:core') testImplementation 'junit:junit:4.13.1' + testImplementation project(path: ':runners:google-cloud-dataflow-java') testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") + testImplementation project(path: ":runners:google-cloud-dataflow-java") + testImplementation project(path: ":sdks:java:extensions:google-cloud-platform-core") + testImplementation project(path: ":sdks:java:io:google-cloud-platform") } \ No newline at end of file diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/CompletedSequenceRange.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ContiguousSequenceRange.java similarity index 65% rename from sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/CompletedSequenceRange.java rename to sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ContiguousSequenceRange.java index 2e04d604231c..b36f4ffa1ea4 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/CompletedSequenceRange.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ContiguousSequenceRange.java @@ -4,36 +4,44 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.io.Serializable; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; -import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.coders.InstantCoder; import org.apache.beam.sdk.coders.VarLongCoder; -import org.apache.beam.sdk.extensions.ordered.CompletedSequenceRange.CompletedSequenceRangeCoder; -import org.apache.beam.sdk.schemas.AutoValueSchema; -import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.checkerframework.checker.initialization.qual.Initialized; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.UnknownKeyFor; import org.joda.time.Instant; +/** + * A range of contiguous event sequences and the latest timestamp of the events in the range. + */ @AutoValue -public abstract class CompletedSequenceRange { -// private static final long serialVersionUID = 1L; - public static final CompletedSequenceRange EMPTY = - CompletedSequenceRange.of(Long.MIN_VALUE, Long.MIN_VALUE, Instant.ofEpochMilli( +public abstract class ContiguousSequenceRange { + public static final ContiguousSequenceRange EMPTY = + ContiguousSequenceRange.of(Long.MIN_VALUE, Long.MIN_VALUE, Instant.ofEpochMilli( Long.MIN_VALUE)); + /** + * @return inclusive starting sequence + */ public abstract long getStart(); + + /** + * @return inclusive end sequence + */ public abstract long getEnd(); + + /** + * @return latest timestamp of all events in the range + */ public abstract Instant getTimestamp(); - public static CompletedSequenceRange of(long start, long end, Instant timestamp) { - return new AutoValue_CompletedSequenceRange(start, end, timestamp); + public static ContiguousSequenceRange of(long start, long end, Instant timestamp) { + return new AutoValue_ContiguousSequenceRange(start, end, timestamp); } - static class CompletedSequenceRangeCoder extends CustomCoder { + static class CompletedSequenceRangeCoder extends CustomCoder { private static final CompletedSequenceRangeCoder INSTANCE = new CompletedSequenceRangeCoder(); @@ -45,7 +53,7 @@ private CompletedSequenceRangeCoder() { } @Override - public void encode(CompletedSequenceRange value, + public void encode(ContiguousSequenceRange value, @UnknownKeyFor @NonNull @Initialized OutputStream outStream) throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { VarLongCoder.of().encode(value.getStart(), outStream); @@ -54,12 +62,12 @@ public void encode(CompletedSequenceRange value, } @Override - public CompletedSequenceRange decode(@UnknownKeyFor @NonNull @Initialized InputStream inStream) + public ContiguousSequenceRange decode(@UnknownKeyFor @NonNull @Initialized InputStream inStream) throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { long start = VarLongCoder.of().decode(inStream); long end = VarLongCoder.of().decode(inStream); Instant timestamp = InstantCoder.of().decode(inStream); - return CompletedSequenceRange.of(start, end, timestamp); + return ContiguousSequenceRange.of(start, end, timestamp); } } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/EventExaminer.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/EventExaminer.java index 1e4fe7565517..15f49546d323 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/EventExaminer.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/EventExaminer.java @@ -31,7 +31,8 @@ public interface EventExaminer> extends Serializable { /** - * Is this event the first expected event for the given key and window? + * Is this event the first expected event for the given key and window if the per key sequence + * is used? In case of global sequence it determines the first global sequence event. * * @param sequenceNumber the sequence number of the event as defined by the key of the input * PCollection to {@link OrderedEventProcessor} @@ -41,8 +42,8 @@ public interface EventExaminer> boolean isInitialEvent(long sequenceNumber, EventT event); /** - * If the event was the first event in the sequence, create the state to hold the required data - * needed for processing. This data will be persisted. + * If the event was the first event for a given key, create the state to hold the required data + * needed for processing. This data will be persisted in a Beam state. * * @param event the first event in the sequence. * @return the state to persist. @@ -53,6 +54,8 @@ public interface EventExaminer> /** * Is this event the last expected event for a given key and window? * + * Note, this method is not used yet with global sequences. + * * @param sequenceNumber of the event * @param event being processed * @return true if the last event. There are cases where it's impossible to know whether it's the diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java index d9547ef766d1..7560eb049fe1 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java @@ -1,52 +1,72 @@ package org.apache.beam.sdk.extensions.ordered; -import org.apache.beam.sdk.extensions.ordered.CompletedSequenceRange.CompletedSequenceRangeCoder; +import org.apache.beam.sdk.extensions.ordered.ContiguousSequenceRange.CompletedSequenceRangeCoder; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.AfterFirst; import org.apache.beam.sdk.transforms.windowing.AfterPane; import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; -import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Repeatedly; import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.transforms.windowing.WindowFn; 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.TimestampedValue; -import org.checkerframework.checker.initialization.qual.Initialized; -import org.checkerframework.checker.nullness.qual.NonNull; -import org.checkerframework.checker.nullness.qual.UnknownKeyFor; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; +/** + * PTransform to produce the side input of the maximum contiguous range of sequence numbers + * + * @param type of event key + * @param type of event + * @param type of processing result + * @param type of state + */ class GlobalSequenceTracker> extends - PTransform>>>, PCollectionView> { + PTransform>>>, PCollectionView> { - private final Combine.GloballyAsSingletonView>>, CompletedSequenceRange> sideInputProducer; + private final Combine.GloballyAsSingletonView>>, ContiguousSequenceRange> sideInputProducer; + private final @Nullable Duration frequencyOfGeneration; public GlobalSequenceTracker( - Combine.GloballyAsSingletonView>>, CompletedSequenceRange> sideInputProducer) { + Combine.GloballyAsSingletonView>>, ContiguousSequenceRange> sideInputProducer) { this.sideInputProducer = sideInputProducer; + this.frequencyOfGeneration = null; + } + + public GlobalSequenceTracker( + Combine.GloballyAsSingletonView>>, ContiguousSequenceRange> sideInputProducer, + Duration globalSequenceGenerationFrequency) { + this.sideInputProducer = sideInputProducer; + this.frequencyOfGeneration = globalSequenceGenerationFrequency; } @Override - public PCollectionView expand( + public PCollectionView expand( PCollection>>> input) { input.getPipeline().getCoderRegistry().registerCoderForClass( - CompletedSequenceRange.class, - CompletedSequenceRangeCoder.of()); + ContiguousSequenceRange.class, CompletedSequenceRangeCoder.of()); + if (frequencyOfGeneration != null) { + input = input + .apply("Triggering Setup", + Window.>>>into( + (WindowFn>>, ?>) + input.getWindowingStrategy().getWindowFn() + ) + .accumulatingFiredPanes() + // TODO: verify that we don't need to have the lateness parameterized + .withAllowedLateness(Duration.ZERO) + .triggering( + Repeatedly.forever(AfterFirst.of( + AfterPane.elementCountAtLeast(1), + AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(frequencyOfGeneration))))); + } return input - // TODO: get the windowing strategy from the input rather than assume global windows. - .apply("Setup Triggering", - Window.>>>into( - new GlobalWindows()) - .accumulatingFiredPanes() - .triggering( - Repeatedly.forever(AfterFirst.of( - AfterPane.elementCountAtLeast(1), - AfterProcessingTime.pastFirstElementInPane() - .plusDelayOf(Duration.standardSeconds(5)))))) .apply("Create Side Input", sideInputProducer); } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java index 863fbca1cd57..8226e1ff5046 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java @@ -20,6 +20,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * Main Stateful DoFn used to process events in the global sequence mode. + * + * @param + * @param + * @param + * @param + */ class GlobalSequencesProcessorDoFn> extends ProcessorDoFn { @@ -53,23 +61,8 @@ class GlobalSequencesProcessorDoFn latestContinuousSequenceSideInput; - - /** - * Stateful DoFn to do the bulk of processing. - * - * @param eventExaminer - * @param eventCoder - * @param stateCoder - * @param keyCoder - * @param mainOutputTupleTag - * @param statusTupleTag - * @param statusUpdateFrequency - * @param unprocessedEventTupleTag - * @param produceStatusUpdateOnEveryEvent - * @param maxNumberOfResultsToProduce - * @param latestContinuousSequenceSideInput - */ + private final PCollectionView latestContiguousRangeSideInput; + GlobalSequencesProcessorDoFn(EventExaminer eventExaminer, Coder eventCoder, Coder stateCoder, @@ -80,12 +73,12 @@ class GlobalSequencesProcessorDoFn>>> unprocessedEventTupleTag, boolean produceStatusUpdateOnEveryEvent, long maxNumberOfResultsToProduce, - PCollectionView latestContinuousSequenceSideInput) { + PCollectionView latestContiguousRangeSideInput) { super(eventExaminer, mainOutputTupleTag, statusTupleTag, statusUpdateFrequency, unprocessedEventTupleTag, produceStatusUpdateOnEveryEvent, maxNumberOfResultsToProduce); - this.latestContinuousSequenceSideInput = latestContinuousSequenceSideInput; + this.latestContiguousRangeSideInput = latestContiguousRangeSideInput; this.bufferedEventsSpec = StateSpecs.orderedList(eventCoder); this.processingStateSpec = StateSpecs.value(ProcessingStateCoder.of(keyCoder)); this.mutableStateSpec = StateSpecs.value(stateCoder); @@ -114,15 +107,15 @@ public void processElement(ProcessContext context, MultiOutputReceiver outputReceiver, BoundedWindow window) { - CompletedSequenceRange lastContinuousSequence = context.sideInput( - latestContinuousSequenceSideInput); + ContiguousSequenceRange lastContiguousRange = context.sideInput( + latestContiguousRangeSideInput); EventT event = eventAndSequence.getValue().getValue(); EventKeyT key = eventAndSequence.getKey(); long sequence = eventAndSequence.getValue().getKey(); if (LOG.isTraceEnabled()) { - LOG.trace(key + ": " + sequence + " lastSequence: " + lastContinuousSequence); + LOG.trace(key + ": " + sequence + " lastRange: " + lastContiguousRange); } ProcessingState processingState = processingStateProxy.read(); @@ -136,7 +129,7 @@ public void processElement(ProcessContext context, } } - processingState.updateGlobalSequenceDetails(lastContinuousSequence); + processingState.updateGlobalSequenceDetails(lastContiguousRange); if (event == null) { // This is a ticker event. We only need to update the state as it relates to the global sequence. @@ -176,7 +169,7 @@ public void processElement(ProcessContext context, private void setBatchEmissionTimerIfNeeded(Timer batchEmissionTimer, ProcessingState processingState) { - CompletedSequenceRange lastCompleteGlobalSequence = processingState.getLastCompleteGlobalSequence(); + ContiguousSequenceRange lastCompleteGlobalSequence = processingState.getLastContiguousRange(); if (lastCompleteGlobalSequence != null && processingState.thereAreGloballySequencedEventsToBeProcessed()) { batchEmissionTimer.set(lastCompleteGlobalSequence.getTimestamp()); @@ -202,8 +195,8 @@ public void onBatchEmission( StateT state = mutableStateState.read(); - CompletedSequenceRange lastCompleteGlobalSequence = processingState.getLastCompleteGlobalSequence(); - if (lastCompleteGlobalSequence == null) { + ContiguousSequenceRange lastContiguousRange = processingState.getLastContiguousRange(); + if (lastContiguousRange == null) { LOG.warn("Last complete global instance is null."); return; } @@ -221,7 +214,7 @@ public void onBatchEmission( this.numberOfResultsBeforeBundleStart = processingState.getResultCount(); state = processBufferedEventRange(processingState, state, bufferedEventsState, outputReceiver, - batchEmissionTimer, lastCompleteGlobalSequence); + batchEmissionTimer, lastContiguousRange); saveStates( processingStatusState, diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java index 1532925c316e..ca61efa08c41 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java @@ -36,6 +36,7 @@ import org.apache.beam.sdk.transforms.ParDo; 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.PCollectionList; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PCollectionView; @@ -43,7 +44,6 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.TypeDescriptor; -import org.joda.time.Duration; import org.joda.time.Instant; /** @@ -134,23 +134,33 @@ public OrderedEventProcessorResult expand( if (handler instanceof OrderedProcessingGlobalSequenceHandler) { OrderedProcessingGlobalSequenceHandler globalSequenceHandler = (OrderedProcessingGlobalSequenceHandler) handler; - final PCollectionView latestContinuousSequence = + + boolean streamingProcessing = input.isBounded() == IsBounded.UNBOUNDED; + + final PCollectionView latestContinuousSequence = input .apply("Convert to SequenceAndTimestamp", ParDo.of(new ToTimestampedEventConverter<>())) .apply("Global Sequence Tracker", - new GlobalSequenceTracker<>(globalSequenceHandler.getGlobalSequenceCombiner())); - - PCollection>> tickers = input.apply("Create Tickers", - new PerKeyTickerGenerator<>(keyCoder, eventCoder, - globalSequenceHandler.getFrequencyOfCheckingForNewGlobalSequence())); - - PCollection>> eventsAndTickers = - PCollectionList.of(input).and(tickers) - .apply("Combine Events and Tickers", Flatten.pCollections()) - .setCoder(tickers.getCoder()); + streamingProcessing ? + new GlobalSequenceTracker<>( + globalSequenceHandler.getGlobalSequenceCombiner(), + globalSequenceHandler.getGlobalSequenceGenerationFrequency()) : + new GlobalSequenceTracker<>( + globalSequenceHandler.getGlobalSequenceCombiner())); + + if (streamingProcessing) { + PCollection>> tickers = input.apply("Create Tickers", + new PerKeyTickerGenerator<>(keyCoder, eventCoder, + globalSequenceHandler.getFrequencyOfCheckingForNewGlobalSequence())); + + input = + PCollectionList.of(input).and(tickers) + .apply("Combine Events and Tickers", Flatten.pCollections()) + .setCoder(tickers.getCoder()); + } processingResult = - eventsAndTickers + input .apply( ParDo.of( new GlobalSequencesProcessorDoFn<>( diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorResult.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorResult.java index c55dc1330d0f..2d4966b3c4a0 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorResult.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorResult.java @@ -52,7 +52,7 @@ public class OrderedEventProcessorResult implements POutp unprocessedEventPCollection; private final TupleTag>>> unprocessedEventTupleTag; - private final @Nullable PCollectionView latestCompletedSequenceRange; + private final @Nullable PCollectionView latestCompletedSequenceRange; OrderedEventProcessorResult( Pipeline pipeline, @@ -75,7 +75,7 @@ public class OrderedEventProcessorResult implements POutp TupleTag> eventProcessingStatusTupleTag, PCollection>>> unprocessedEventPCollection, TupleTag>>> unprocessedEventTupleTag, - @Nullable PCollectionView latestCompletedSequenceRange) { + @Nullable PCollectionView latestCompletedSequenceRange) { this.pipeline = pipeline; this.outputPCollection = outputPCollection; @@ -129,7 +129,7 @@ public PCollection>>> unprocessedEven return unprocessedEventPCollection; } - public @Nullable PCollectionView latestCompletedSequenceRange() { + public @Nullable PCollectionView latestCompletedSequenceRange() { return latestCompletedSequenceRange; } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java index 786b3bd87fb3..3ff4416c57bc 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java @@ -236,7 +236,7 @@ public OrderedProcessingGlobalSequenceHandler( super(eventTClass, keyTClass, stateTClass, resultTClass); } - public GloballyAsSingletonView>>, CompletedSequenceRange> getGlobalSequenceCombiner() { + public GloballyAsSingletonView>>, ContiguousSequenceRange> getGlobalSequenceCombiner() { return Combine.globally( new DefaultSequenceCombiner(getEventExaminer())).asSingletonView(); } @@ -244,5 +244,9 @@ public GloballyAsSingletonView>>, Com public Duration getFrequencyOfCheckingForNewGlobalSequence() { return Duration.standardSeconds(1); } + + public Duration getGlobalSequenceGenerationFrequency() { + return Duration.standardSeconds(1); + } } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java index fc41164cedf6..26e956e1d13e 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java @@ -51,7 +51,7 @@ class ProcessingState { private long resultCount; - @Nullable private CompletedSequenceRange lastCompleteGlobalSequence; + @Nullable private ContiguousSequenceRange lastCompleteGlobalSequence; private KeyT key; @@ -133,11 +133,11 @@ public KeyT getKey() { return key; } - public @Nullable CompletedSequenceRange getLastCompleteGlobalSequence() { + public @Nullable ContiguousSequenceRange getLastContiguousRange() { return lastCompleteGlobalSequence; } - public void setLastCompleteGlobalSequence(@Nullable CompletedSequenceRange lastCompleteGlobalSequence) { + public void setLastCompleteGlobalSequence(@Nullable ContiguousSequenceRange lastCompleteGlobalSequence) { this.lastCompleteGlobalSequence = lastCompleteGlobalSequence; } @@ -301,7 +301,7 @@ public long resultsProducedInBundle(long numberOfResultsBeforeBundleStart) { return resultCount - numberOfResultsBeforeBundleStart; } - public void updateGlobalSequenceDetails(CompletedSequenceRange updated) { + public void updateGlobalSequenceDetails(ContiguousSequenceRange updated) { if(thereAreGloballySequencedEventsToBeProcessed()) { // We don't update the timer if we can already process events in the onTimer batch. // Otherwise, it's possible that we will be pushing the timer to later timestamps @@ -331,8 +331,8 @@ static class ProcessingStateCoder extends Coder> { private static final VarIntCoder INTEGER_CODER = VarIntCoder.of(); private static final BooleanCoder BOOLEAN_CODER = BooleanCoder.of(); - private static final NullableCoder SEQUENCE_AND_TIMESTAMP_CODER = - NullableCoder.of(CompletedSequenceRange.CompletedSequenceRangeCoder.of()); + private static final NullableCoder SEQUENCE_AND_TIMESTAMP_CODER = + NullableCoder.of(ContiguousSequenceRange.CompletedSequenceRangeCoder.of()); private Coder keyCoder; @@ -355,7 +355,7 @@ public void encode(ProcessingState value, OutputStream outStream) throws I LONG_CODER.encode(value.getResultCount(), outStream); BOOLEAN_CODER.encode(value.isLastEventReceived(), outStream); keyCoder.encode(value.getKey(), outStream); - SEQUENCE_AND_TIMESTAMP_CODER.encode(value.getLastCompleteGlobalSequence(), outStream); + SEQUENCE_AND_TIMESTAMP_CODER.encode(value.getLastContiguousRange(), outStream); } @Override @@ -369,7 +369,7 @@ public ProcessingState decode(InputStream inStream) throws IOException { long resultCount = LONG_CODER.decode(inStream); boolean isLastEventReceived = BOOLEAN_CODER.decode(inStream); KeyT key = keyCoder.decode(inStream); - CompletedSequenceRange lastCompleteGlobalSequence = SEQUENCE_AND_TIMESTAMP_CODER.decode(inStream); + ContiguousSequenceRange lastCompleteGlobalSequence = SEQUENCE_AND_TIMESTAMP_CODER.decode(inStream); ProcessingState result = new ProcessingState<>( key, diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java index 710a5353a15f..5b24956bb8dd 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java @@ -286,7 +286,7 @@ private void bufferEvent(long currentSequence, EventT currentEvent, StateT processBufferedEventRange(ProcessingState processingState, @Nullable StateT state, OrderedListState bufferedEventsState, MultiOutputReceiver outputReceiver, - Timer largeBatchEmissionTimer, CompletedSequenceRange completedSequenceRange) { + Timer largeBatchEmissionTimer, ContiguousSequenceRange contiguousSequenceRange) { Long earliestBufferedSequence = processingState.getEarliestBufferedSequence(); Long latestBufferedSequence = processingState.getLatestBufferedSequence(); if (earliestBufferedSequence == null || latestBufferedSequence == null) { @@ -310,7 +310,7 @@ StateT processBufferedEventRange(ProcessingState processingState, EventT bufferedEvent = timestampedEvent.getValue(); boolean skipProcessing = false; - if (completedSequenceRange != null && eventSequence < completedSequenceRange.getStart()) { + if (contiguousSequenceRange != null && eventSequence < contiguousSequenceRange.getStart()) { // In case of global sequence processing - remove the elements below the range start skipProcessing = true; endClearRange = fromLong(eventSequence); @@ -341,7 +341,7 @@ StateT processBufferedEventRange(ProcessingState processingState, lastOutputSequence != null && eventSequence == lastOutputSequence + 1; boolean continueProcessing = checkForSequenceGapInBufferedEvents() ? currentEventIsNextInSequence : - (eventSequence <= completedSequenceRange.getEnd() || currentEventIsNextInSequence); + (eventSequence <= contiguousSequenceRange.getEnd() || currentEventIsNextInSequence); if (!continueProcessing) { processingState.foundSequenceGap(eventSequence); // Records will be cleared up to this element diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java index 4d073dc204ba..cb23d680a774 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java @@ -193,7 +193,7 @@ private void processBufferedEvents( } processBufferedEventRange(processingState, state, bufferedEventsState, outputReceiver, - largeBatchEmissionTimer, CompletedSequenceRange.EMPTY); + largeBatchEmissionTimer, ContiguousSequenceRange.EMPTY); } @OnTimer(LARGE_BATCH_EMISSION_TIMER) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java index a894836b8640..4d98c3b61ace 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java @@ -5,7 +5,7 @@ import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; -import org.apache.beam.sdk.extensions.ordered.CompletedSequenceRange; +import org.apache.beam.sdk.extensions.ordered.ContiguousSequenceRange; import org.apache.beam.sdk.extensions.ordered.EventExaminer; import org.apache.beam.sdk.extensions.ordered.MutableState; import org.apache.beam.sdk.extensions.ordered.combiner.SequenceRangeAccumulator.SequenceRangeAccumulatorCoder; @@ -20,8 +20,20 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * Default global sequence combiner. + *

    + * Produces {@link ContiguousSequenceRange} of contiguous longs starting from the initial event + * identified by {@link EventExaminer#isInitialEvent(long, EventT)}. + *

    + * This combiner currently doesn't use {@link EventExaminer#isLastEvent(long, EventT)}. + * + * @param type of key + * @param type of event + * @param type of state + */ public class DefaultSequenceCombiner> extends - CombineFn>>, SequenceRangeAccumulator, CompletedSequenceRange> { + CombineFn>>, SequenceRangeAccumulator, ContiguousSequenceRange> { private static final Logger LOG = LoggerFactory.getLogger(DefaultSequenceCombiner.class); @@ -67,9 +79,9 @@ public SequenceRangeAccumulator mergeAccumulators( } @Override - public CompletedSequenceRange extractOutput(SequenceRangeAccumulator accum) { - CompletedSequenceRange result = accum.largestContinuousRange(); - if(LOG.isTraceEnabled()) { + public ContiguousSequenceRange extractOutput(SequenceRangeAccumulator accum) { + ContiguousSequenceRange result = accum.largestContinuousRange(); + if (LOG.isTraceEnabled()) { LOG.trace("Returning completed sequence range: " + result); } return result; @@ -80,6 +92,6 @@ public CompletedSequenceRange extractOutput(SequenceRangeAccumulator accum) { @UnknownKeyFor @NonNull @Initialized CoderRegistry registry, @UnknownKeyFor @NonNull @Initialized Coder>>> inputCoder) throws @UnknownKeyFor @NonNull @Initialized CannotProvideCoderException { - return new SequenceRangeAccumulatorCoder(); + return SequenceRangeAccumulatorCoder.of(); } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java index 63c8dcf0be96..19df464e584e 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java @@ -13,18 +13,19 @@ import org.apache.beam.sdk.coders.NullableCoder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; -import org.apache.beam.sdk.extensions.ordered.CompletedSequenceRange; +import org.apache.beam.sdk.extensions.ordered.ContiguousSequenceRange; import org.apache.commons.lang3.tuple.Pair; import org.checkerframework.checker.initialization.qual.Initialized; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.UnknownKeyFor; import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +/** + * Default accumulator used to combine sequence ranges. + */ public class SequenceRangeAccumulator { - static Instant max(Instant a, Instant b) { + private static Instant max(Instant a, Instant b) { return a.isAfter(b) ? a : b; } @@ -91,9 +92,9 @@ private void clearRangesBelowInitialSequence(long sequence, Instant timestamp) { data.subMap(Long.MIN_VALUE, sequence).clear(); } - public CompletedSequenceRange largestContinuousRange() { + public ContiguousSequenceRange largestContinuousRange() { if (initialSequence == null) { - return CompletedSequenceRange.EMPTY; + return ContiguousSequenceRange.EMPTY; } Entry> firstEntry = data.firstEntry(); @@ -103,7 +104,7 @@ public CompletedSequenceRange largestContinuousRange() { Long start = firstEntry.getKey(); Long end = firstEntry.getValue().getLeft(); Instant latestTimestamp = firstEntry.getValue().getRight(); - return CompletedSequenceRange.of(start, end, latestTimestamp); + return ContiguousSequenceRange.of(start, end, latestTimestamp); } public int numberOfRanges() { @@ -113,7 +114,7 @@ public int numberOfRanges() { public void merge(SequenceRangeAccumulator another) { if (this.initialSequence != null && another.initialSequence != null - && ! this.initialSequence.equals(another.initialSequence)) { + && !this.initialSequence.equals(another.initialSequence)) { throw new IllegalStateException("Two accumulators contain different initial sequences: " + this.initialSequence + " and " + another.initialSequence); } @@ -206,6 +207,15 @@ public String toString() { public static class SequenceRangeAccumulatorCoder extends CustomCoder { + private static final SequenceRangeAccumulatorCoder INSTANCE = new SequenceRangeAccumulatorCoder(); + + public static SequenceRangeAccumulatorCoder of() { + return INSTANCE; + } + + private SequenceRangeAccumulatorCoder() { + } + private final NullableCoder initialSequenceCoder = NullableCoder.of(VarLongCoder.of()); private final VarIntCoder numberOfRangesCoder = VarIntCoder.of(); private final VarLongCoder dataCoder = VarLongCoder.of(); diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java index b575cd55cd2a..66618364c345 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java @@ -4,6 +4,8 @@ import java.util.Arrays; import java.util.Collection; import java.util.List; +import java.util.logging.Level; +import java.util.logging.Logger; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.extensions.ordered.StringBufferOrderedProcessingHandler.StringBufferOrderedProcessingWithGlobalSequenceHandler; @@ -25,6 +27,11 @@ public class OrderedEventProcessorGlobalSequenceTest extends OrderedEventProcess public static final boolean GLOBAL_SEQUENCE = true; + static { + Logger logger = Logger.getLogger(GlobalSequencesProcessorDoFn.class.getName()); + logger.setLevel(Level.FINEST); + } + @org.junit.Test public void testPerfectOrderingProcessing() throws CannotProvideCoderException { Event[] events = { @@ -50,7 +57,7 @@ public void testPerfectOrderingProcessing() throws CannotProvideCoderException { EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, LARGE_MAX_RESULTS_PER_OUTPUT, - CompletedSequenceRange.of(0,5, new Instant())); + ContiguousSequenceRange.of(0, 5, new Instant())); } @Test @@ -84,7 +91,7 @@ public void testOutOfSequenceProcessing() throws CannotProvideCoderException { EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, LARGE_MAX_RESULTS_PER_OUTPUT, - CompletedSequenceRange.of(0,8, new Instant())); + ContiguousSequenceRange.of(0, 8, new Instant())); } @Test @@ -124,7 +131,7 @@ public void testHandlingOfDuplicateSequences() throws CannotProvideCoderExceptio EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, LARGE_MAX_RESULTS_PER_OUTPUT, - CompletedSequenceRange.of(0,3, new Instant())); + ContiguousSequenceRange.of(0, 3, new Instant())); } @Test @@ -158,7 +165,7 @@ public void testTreatingSequencesBelowInitialAsDuplicates() throws CannotProvide EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, LARGE_MAX_RESULTS_PER_OUTPUT, - CompletedSequenceRange.of(0,3, new Instant())); + ContiguousSequenceRange.of(0, 3, new Instant())); } @Test @@ -196,7 +203,7 @@ public void testHandlingOfCheckedExceptions() throws CannotProvideCoderException // Sequence matcher doesn't know if the element is valid or not. // That's why the elements that are get rejected in the processor still count when // calculating the global sequence - CompletedSequenceRange.of(0,3, new Instant())); + ContiguousSequenceRange.of(0, 3, new Instant())); } @Test @@ -223,7 +230,7 @@ public void testProcessingWithEveryOtherResultEmission() throws CannotProvideCod EMISSION_FREQUENCY_ON_EVERY_OTHER_EVENT, INITIAL_SEQUENCE_OF_0, LARGE_MAX_RESULTS_PER_OUTPUT, - CompletedSequenceRange.of(0,5, new Instant())); + ContiguousSequenceRange.of(0, 5, new Instant())); } @Test @@ -258,7 +265,7 @@ public void testLargeBufferedOutputInTimer() throws CannotProvideCoderException EMISSION_FREQUENCY_ON_EVERY_ELEMENT, 1L /* This dataset assumes 1 as the starting sequence */, maxResultsPerOutput, - CompletedSequenceRange.of(1, sequences.length, new Instant())); + ContiguousSequenceRange.of(1, sequences.length, new Instant())); } @Test @@ -289,7 +296,7 @@ public void testSequenceGapProcessingInBufferedOutput() throws CannotProvideCode EMISSION_FREQUENCY_ON_EVERY_ELEMENT, 1L /* This dataset assumes 1 as the starting sequence */, maxResultsPerOutput, - CompletedSequenceRange.of(1,10, new Instant())); + ContiguousSequenceRange.of(1, 10, new Instant())); } @Test @@ -322,7 +329,7 @@ public void testHandlingOfMaxSequenceNumber() throws CannotProvideCoderException EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, LARGE_MAX_RESULTS_PER_OUTPUT, - CompletedSequenceRange.of(0, 2, Instant.now())); + ContiguousSequenceRange.of(0, 2, Instant.now())); } @Test @@ -345,7 +352,7 @@ public void testProcessingOfTheLastInput() throws CannotProvideCoderException { EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, LARGE_MAX_RESULTS_PER_OUTPUT, - CompletedSequenceRange.of(0,2, new Instant())); + ContiguousSequenceRange.of(0, 2, new Instant())); } @@ -355,7 +362,7 @@ private void testGlobalSequenceProcessing( int emissionFrequency, long initialSequence, int maxResultsPerOutput, - CompletedSequenceRange expectedLastCompleteRange) + ContiguousSequenceRange expectedLastCompleteRange) throws CannotProvideCoderException { testGlobalSequenceProcessing( events, @@ -373,38 +380,43 @@ private void testGlobalSequenceProcessing( int emissionFrequency, long initialSequence, int maxResultsPerOutput, - CompletedSequenceRange expectedLastCompleteRange) + ContiguousSequenceRange expectedLastCompleteRange) throws CannotProvideCoderException { // Test a streaming pipeline - doTest( - events, - null /* expectedStatuses */, - expectedOutput, - expectedUnprocessedEvents, - emissionFrequency, - initialSequence, - maxResultsPerOutput, - false /* produceStatusOnEveryEvent */, - STREAMING, - GLOBAL_SEQUENCE, expectedLastCompleteRange); - - if (true) { - // TODO: Test batch processing - return; + if (false) { + doTest( + events, + null /* expectedStatuses */, + expectedOutput, + expectedUnprocessedEvents, + emissionFrequency, + initialSequence, + maxResultsPerOutput, + false /* produceStatusOnEveryEvent */, + STREAMING, + GLOBAL_SEQUENCE, expectedLastCompleteRange); } + // Test a batch pipeline - doTest( - events, - null /* expectedStatuses */, - expectedOutput, - expectedUnprocessedEvents, - emissionFrequency, - initialSequence, - maxResultsPerOutput, - false /* produceStatusOnEveryEvent */, - BATCH, - GLOBAL_SEQUENCE, - expectedLastCompleteRange); + if (runTestsOnDataflowRunner()) { + doTest( + events, + null /* expectedStatuses */, + expectedOutput, + expectedUnprocessedEvents, + emissionFrequency, + initialSequence, + maxResultsPerOutput, + false /* produceStatusOnEveryEvent */, + BATCH, + GLOBAL_SEQUENCE, + expectedLastCompleteRange); + } else { + System.err.println( + "Warning - batch tests didn't run. " + + "DirectRunner doesn't work correctly with this transform in batch mode." + + "Run the tests using Dataflow runner to validate."); + } } diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java index a3e98275636c..818063015825 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java @@ -565,7 +565,7 @@ protected void testPerKeySequenceProcessing( maxResultsPerOutput, produceStatusOnEveryEvent, STREAMING, - false, CompletedSequenceRange.EMPTY); + false, ContiguousSequenceRange.EMPTY); // Test a batch pipeline doTest( @@ -578,7 +578,7 @@ protected void testPerKeySequenceProcessing( maxResultsPerOutput, produceStatusOnEveryEvent, BATCH, - false, CompletedSequenceRange.EMPTY); + false, ContiguousSequenceRange.EMPTY); } @Test diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTestBase.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTestBase.java index 21ce703e890b..1dd49edd4f1e 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTestBase.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTestBase.java @@ -25,6 +25,8 @@ import java.util.Iterator; import java.util.Set; import javax.annotation.Nullable; +import org.apache.beam.runners.dataflow.TestDataflowPipelineOptions; +import org.apache.beam.runners.dataflow.TestDataflowRunner; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; @@ -47,6 +49,7 @@ import org.apache.beam.sdk.transforms.windowing.Window; 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.checkerframework.checker.initialization.qual.Initialized; import org.checkerframework.checker.nullness.qual.NonNull; @@ -79,6 +82,18 @@ public class OrderedEventProcessorTestBase { @Rule public final transient TestPipeline batchPipeline = TestPipeline.create(); + protected boolean runTestsOnDataflowRunner() { + return Boolean.getBoolean("run-tests-on-dataflow"); + } + + protected String getSystemProperty(String name) { + String property = System.getProperty(name); + if(property == null) { + throw new IllegalStateException("Unable to find system property '" + name + "'"); + } + return property; + } + static class MapEventsToKV extends DoFn>> { @ProcessElement @@ -115,11 +130,16 @@ protected void doTest( boolean produceStatusOnEveryEvent, boolean streaming, boolean isGlobalSequence, - @Nullable CompletedSequenceRange expectedLastCompletedSequence) + @Nullable ContiguousSequenceRange expectedLastCompletedSequence) throws @UnknownKeyFor @NonNull @Initialized CannotProvideCoderException { Pipeline pipeline = streaming ? streamingPipeline : batchPipeline; - + if (runTestsOnDataflowRunner()) { + pipeline.getOptions().setRunner(TestDataflowRunner.class); + TestDataflowPipelineOptions options = pipeline.getOptions().as(TestDataflowPipelineOptions.class); + options.setExperiments(Arrays.asList("disable_runner_v2")); + options.setTempRoot("gs://" + getSystemProperty("temp_dataflow_bucket")); + } PCollection rawInput = streaming ? createStreamingPCollection(pipeline, events) @@ -187,7 +207,7 @@ protected void doTest( if (expectedLastCompletedSequence != null && processingResult.latestCompletedSequenceRange() != null) { - PCollection globalSequences = rawInput + PCollection globalSequences = rawInput .apply("Publish Global Sequences", new GlobalSequenceRangePublisher(processingResult.latestCompletedSequenceRange(), handler.getKeyCoder(pipeline, input.getCoder()), @@ -200,21 +220,21 @@ protected void doTest( } static class LastExpectedGlobalSequenceRangeMatcher implements - SerializableFunction, Void> { + SerializableFunction, Void> { private final long expectedStart; private final long expectedEnd; - LastExpectedGlobalSequenceRangeMatcher(CompletedSequenceRange expected) { + LastExpectedGlobalSequenceRangeMatcher(ContiguousSequenceRange expected) { this.expectedStart = expected.getStart(); this.expectedEnd = expected.getEnd(); } @Override - public Void apply(Iterable input) { + public Void apply(Iterable input) { StringBuilder listOfRanges = new StringBuilder("["); - Iterator iterator = input.iterator(); - CompletedSequenceRange lastRange = null; + Iterator iterator = input.iterator(); + ContiguousSequenceRange lastRange = null; while (iterator.hasNext()) { lastRange = iterator.next(); @@ -228,7 +248,8 @@ public Void apply(Iterable input) { lastRange.getStart() == expectedStart && lastRange.getEnd() == expectedEnd; assertThat( - "Expected range not found: [" + expectedStart + '-' + expectedEnd + "], received ranges: " + "Expected range not found: [" + expectedStart + '-' + expectedEnd + + "], received ranges: " + listOfRanges, foundExpectedRange); return null; } @@ -311,14 +332,14 @@ static String normalizeExplanation(String value) { } static class GlobalSequenceRangePublisher extends - PTransform, PCollection> { + PTransform, PCollection> { - private final PCollectionView lastCompletedSequenceRangeView; + private final PCollectionView lastCompletedSequenceRangeView; private final Coder keyCoder; private final Coder eventCoder; public GlobalSequenceRangePublisher( - PCollectionView latestCompletedSequenceRange, + PCollectionView latestCompletedSequenceRange, Coder keyCoder, Coder eventCoder) { this.lastCompletedSequenceRangeView = latestCompletedSequenceRange; this.keyCoder = keyCoder; @@ -326,30 +347,38 @@ public GlobalSequenceRangePublisher( } @Override - public PCollection expand(PCollection input) { - return - input - // In production pipelines the global sequence will typically be obtained - // by using GenerateSequence. But GenerateSequence doesn't work well with TestStream, - // That's why we use the input events here. + public PCollection expand(PCollection input) { + PCollection>> events = input + // In production pipelines the global sequence will typically be obtained + // by using GenerateSequence. But GenerateSequence doesn't work well with TestStream, + // That's why we use the input events here. // .apply("Create Ticker", // GenerateSequence.from(0).to(2).withRate(1, Duration.standardSeconds(5))) - .apply("To KV", ParDo.of(new MapEventsToKV())) - .apply("Create Tickers", - new PerKeyTickerGenerator<>(keyCoder, eventCoder, - Duration.standardSeconds(1))) - .apply("Emit SideInput", ParDo.of(new SideInputEmitter()) - .withSideInput("lastCompletedSequence", lastCompletedSequenceRangeView)); + .apply("To KV", ParDo.of(new MapEventsToKV())); + if (input.isBounded() == IsBounded.BOUNDED) { + return events.apply("Emit SideInput", ParDo.of(new SideInputEmitter()) + .withSideInput("lastCompletedSequence", lastCompletedSequenceRangeView)); + } else { + PCollection>> tickers = events + .apply("Create Tickers", + new PerKeyTickerGenerator<>(keyCoder, eventCoder, + Duration.standardSeconds(1))); + return + tickers + .apply("Emit SideInput", ParDo.of(new SideInputEmitter()) + .withSideInput("lastCompletedSequence", lastCompletedSequenceRangeView)); + } } - } - static class SideInputEmitter extends DoFn>, CompletedSequenceRange> { + static class SideInputEmitter extends + DoFn>, ContiguousSequenceRange> { - @ProcessElement - public void produceCompletedRange( - @SideInput("lastCompletedSequence") CompletedSequenceRange sideInput, - OutputReceiver outputReceiver) { - outputReceiver.output(sideInput); + @ProcessElement + public void produceCompletedRange( + @SideInput("lastCompletedSequence") ContiguousSequenceRange sideInput, + OutputReceiver outputReceiver) { + outputReceiver.output(sideInput); + } } } -} +} \ No newline at end of file diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorCoderTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorCoderTest.java index 7cd3fa8ed613..d2f0a7127818 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorCoderTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorCoderTest.java @@ -11,7 +11,7 @@ public class SequenceRangeAccumulatorCoderTest { - private SequenceRangeAccumulatorCoder coder = new SequenceRangeAccumulatorCoder(); + private SequenceRangeAccumulatorCoder coder = SequenceRangeAccumulatorCoder.of(); @Test public void testEncodingEmptyAccumulator() throws IOException { diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java index 133773df66ca..ff3b15142a58 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java @@ -2,7 +2,7 @@ import java.util.Arrays; import java.util.concurrent.atomic.AtomicLong; -import org.apache.beam.sdk.extensions.ordered.CompletedSequenceRange; +import org.apache.beam.sdk.extensions.ordered.ContiguousSequenceRange; import org.joda.time.Instant; import org.junit.Assert; import org.junit.Test; @@ -50,7 +50,7 @@ public void testSimpleAccumulation() { new Event(3, nextTimestamp()) }; - doTestAccumulation(events, CompletedSequenceRange.of(1, 3, eventTimestamp(events, 3)), 1); + doTestAccumulation(events, ContiguousSequenceRange.of(1, 3, eventTimestamp(events, 3)), 1); } @Test @@ -62,7 +62,7 @@ public void testReverseArrivalHandling() { }; Instant timestampOfEventNumber1 = eventTimestamp(events, 1); - doTestAccumulation(events, CompletedSequenceRange.of(1, 3, timestampOfEventNumber1), 1); + doTestAccumulation(events, ContiguousSequenceRange.of(1, 3, timestampOfEventNumber1), 1); } @Test @@ -75,7 +75,7 @@ public void testPartialRangeAccumulation() { new Event(7, nextTimestamp()), }; - doTestAccumulation(events, CompletedSequenceRange.of(1, 3, eventTimestamp(events, 3)), 3); + doTestAccumulation(events, ContiguousSequenceRange.of(1, 3, eventTimestamp(events, 3)), 3); } @Test @@ -89,7 +89,7 @@ public void testMergingRangeAccumulation() { new Event(6, nextTimestamp()), }; - doTestAccumulation(events, CompletedSequenceRange.of(1, 3, eventTimestamp(events, 3)), 2); + doTestAccumulation(events, ContiguousSequenceRange.of(1, 3, eventTimestamp(events, 3)), 2); } @Test @@ -102,14 +102,14 @@ public void testNoStartEvent() { new Event(5, nextTimestamp()), }; - doTestAccumulation(events, CompletedSequenceRange.EMPTY, 2); + doTestAccumulation(events, ContiguousSequenceRange.EMPTY, 2); } @Test public void testNoEventsAccumulation() { Event[] events = new Event[]{}; - doTestAccumulation(events, CompletedSequenceRange.EMPTY, 0); + doTestAccumulation(events, ContiguousSequenceRange.EMPTY, 0); } @Test @@ -128,7 +128,7 @@ public void testRemovingRangesBelowInitialSequenceDuringAccumulation() { new Event(7, nextTimestamp(), true), }; - doTestAccumulation(events, CompletedSequenceRange.of(7, 7, eventTimestamp(events, 7)), 1); + doTestAccumulation(events, ContiguousSequenceRange.of(7, 7, eventTimestamp(events, 7)), 1); } @Test @@ -151,10 +151,10 @@ public void testRemovingElementsBelowInitialSequenceDuringAccumulation() { }; Instant timestampOfTheLastEvent = events[events.length - 1].timestamp; - doTestAccumulation(events, CompletedSequenceRange.of(7, 8, timestampOfTheLastEvent), 1); + doTestAccumulation(events, ContiguousSequenceRange.of(7, 8, timestampOfTheLastEvent), 1); } - private static void doTestAccumulation(Event[] events, CompletedSequenceRange expectedResult, + private static void doTestAccumulation(Event[] events, ContiguousSequenceRange expectedResult, int expectedNumberOfRanges) { SequenceRangeAccumulator accumulator = new SequenceRangeAccumulator(); Arrays.stream(events).forEach(e -> accumulator.add(e.sequence, e.timestamp, e.initialEvent)); @@ -172,7 +172,7 @@ public void testEmptyMerge() { Event[] set1 = new Event[]{}; Event[] set2 = new Event[]{}; - CompletedSequenceRange expectedResult = CompletedSequenceRange.EMPTY; + ContiguousSequenceRange expectedResult = ContiguousSequenceRange.EMPTY; int expectedNumberOfRanges = 0; doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); @@ -187,7 +187,7 @@ public void testMergingNonEmptyWithEmpty() { }; Event[] set2 = new Event[]{}; - CompletedSequenceRange expectedResult = CompletedSequenceRange.of(1, 3, + ContiguousSequenceRange expectedResult = ContiguousSequenceRange.of(1, 3, eventTimestamp(set1, 1L)); int expectedNumberOfRanges = 1; @@ -206,7 +206,7 @@ public void testMergingWithLowerNonAdjacentRange() { new Event(6, nextTimestamp()) }; - CompletedSequenceRange expectedResult = CompletedSequenceRange.of(1, 2, + ContiguousSequenceRange expectedResult = ContiguousSequenceRange.of(1, 2, eventTimestamp(set1, 2L)); int expectedNumberOfRanges = 2; @@ -225,7 +225,7 @@ public void testMergingWithoutAnyInitialEvents() { new Event(6, nextTimestamp()) }; - CompletedSequenceRange expectedResult = CompletedSequenceRange.EMPTY; + ContiguousSequenceRange expectedResult = ContiguousSequenceRange.EMPTY; int expectedNumberOfRanges = 2; doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); @@ -244,7 +244,7 @@ public void testMergingAdjacentRanges() { new Event(6, nextTimestamp()) }; - CompletedSequenceRange expectedResult = CompletedSequenceRange.of(1, 6, + ContiguousSequenceRange expectedResult = ContiguousSequenceRange.of(1, 6, eventTimestamp(set2, 6L)); int expectedNumberOfRanges = 1; @@ -264,7 +264,7 @@ public void testPruningSequencesBelowInitial() { new Event(6, nextTimestamp()) }; - CompletedSequenceRange expectedResult = CompletedSequenceRange.of(3, 6, + ContiguousSequenceRange expectedResult = ContiguousSequenceRange.of(3, 6, eventTimestamp(set2, 6L)); int expectedNumberOfRanges = 1; @@ -286,7 +286,7 @@ public void testDuplicateHandling() { new Event(6, nextTimestamp()) }; - CompletedSequenceRange expectedResult = CompletedSequenceRange.of(1, 6, + ContiguousSequenceRange expectedResult = ContiguousSequenceRange.of(1, 6, eventTimestamp(set2, 6L)); int expectedNumberOfRanges = 1; @@ -307,7 +307,7 @@ public void testExceptionThrownIfThereAreDifferentInitialSequences() { }; try { - doTestMerging(set1, set2, CompletedSequenceRange.EMPTY, 0); + doTestMerging(set1, set2, ContiguousSequenceRange.EMPTY, 0); Assert.fail("Expected to throw an exception"); } catch (IllegalStateException e) { Assert.assertEquals("Exception message", @@ -329,14 +329,14 @@ public void testSelectingHighestTimestampWhenMerging() { new Event(6, nextTimestamp()) }; - CompletedSequenceRange expectedResult = CompletedSequenceRange.of(1, 6, + ContiguousSequenceRange expectedResult = ContiguousSequenceRange.of(1, 6, eventTimestamp(set1, 2L)); int expectedNumberOfRanges = 1; doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); } private static void doTestMerging(Event[] set1, Event[] set2, - CompletedSequenceRange expectedResult, + ContiguousSequenceRange expectedResult, int expectedNumberOfRanges) { // Try to merge both set2 to set1 and set1 to set2 - both must return the same results mergeAndTest(set1, set2, expectedResult, expectedNumberOfRanges, "set1"); @@ -344,7 +344,7 @@ private static void doTestMerging(Event[] set1, Event[] set2, } private static void mergeAndTest(Event[] set1, Event[] set2, - CompletedSequenceRange expectedResult, + ContiguousSequenceRange expectedResult, int expectedNumberOfRanges, String firstSetName) { final SequenceRangeAccumulator a1 = new SequenceRangeAccumulator(); Arrays.stream(set1).forEach(e -> a1.add(e.sequence, e.timestamp, e.initialEvent)); diff --git a/sdks/java/extensions/ordered/src/test/resources/logging.properties b/sdks/java/extensions/ordered/src/test/resources/logging.properties index 7e6f820baafd..64edb0a5fa9b 100644 --- a/sdks/java/extensions/ordered/src/test/resources/logging.properties +++ b/sdks/java/extensions/ordered/src/test/resources/logging.properties @@ -9,7 +9,7 @@ java.util.logging.SimpleFormatter.format = [%1$tc] %4$s: %2$s - %5$s %6$s%n # OFF, SEVERE, WARNING, INFO, CONFIG, FINE, FINER, FINEST, ALL # root logger -.level = TRACE +.level = FINEST # child logger org.example.level = ALL \ No newline at end of file From fa1654731e930a2eacd1283d470f2b724601281c Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Mon, 23 Sep 2024 15:06:56 -0700 Subject: [PATCH 18/33] A round of documentation update and minor clean up. --- CHANGES.md | 1 + .../ordered/GlobalSequenceTracker.java | 8 +- .../ordered/OrderedEventProcessor.java | 221 +++++++++++------- .../ordered/OrderedEventProcessorResult.java | 13 +- .../ordered/OrderedProcessingHandler.java | 29 +++ .../ordered/PerKeyTickerGenerator.java | 8 +- .../ordered/SequencePerKeyProcessorDoFn.java | 8 + .../sdk/extensions/ordered/package-info.java | 3 +- .../OrderedEventProcessorTestBase.java | 4 +- 9 files changed, 195 insertions(+), 100 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index d58ceffeb411..0d2e2d3a7b50 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -68,6 +68,7 @@ * Dataflow worker can install packages from Google Artifact Registry Python repositories (Python) ([#32123](https://github.com/apache/beam/issues/32123)). * Added support for Zstd codec in SerializableAvroCodecFactory (Java) ([#32349](https://github.com/apache/beam/issues/32349)) +* Added support for processing events which use a global sequence to OrderedEventProcessor (Java) * X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). ## Breaking Changes diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java index 7560eb049fe1..05b6d9d09f89 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java @@ -50,8 +50,12 @@ public PCollectionView expand( ContiguousSequenceRange.class, CompletedSequenceRangeCoder.of()); if (frequencyOfGeneration != null) { + // This branch will only be executed in case of streaming pipelines. + // For batch pipelines the side input should only be computed once. input = input .apply("Triggering Setup", + // Reproduce the windowing of the input PCollection, but change the triggering + // in order to create a slowing changing side input Window.>>>into( (WindowFn>>, ?>) input.getWindowingStrategy().getWindowFn() @@ -65,8 +69,6 @@ public PCollectionView expand( AfterProcessingTime.pastFirstElementInPane() .plusDelayOf(frequencyOfGeneration))))); } - return - input - .apply("Create Side Input", sideInputProducer); + return input.apply("Create Side Input", sideInputProducer); } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java index ca61efa08c41..96b0db50c019 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java @@ -50,10 +50,13 @@ * Transform for processing ordered events. Events are grouped by the key and within each key they * are applied according to the provided sequence. Events which arrive out of sequence are buffered * and processed after all the missing events for a given key have arrived. + *

    + * There are two sequencing modes - a sequence per key and a global sequence. See + * {@link OrderedProcessingHandler} for details on how to configure this transform. * - * @param - * @param - * @param + * @param type of event + * @param type of event key + * @param type of the state */ @AutoValue @SuppressWarnings({"nullness", "TypeNameShadowing"}) @@ -65,6 +68,16 @@ public abstract class OrderedEventProcessor< public static final String GLOBAL_SEQUENCE_TRACKER = "global_sequence_tracker"; + /** + * Create the transform. + * + * @param handler provides the configuration of this transform + * @param type of event + * @param type of event key + * @param type of the result object + * @param type of the state to store + * @return the transform + */ public static < EventTypeT, EventKeyTypeT, @@ -130,95 +143,127 @@ public OrderedEventProcessorResult expand( KvCoder.of( keyCoder, KvCoder.of(VarLongCoder.of(), new UnprocessedEventCoder<>(eventCoder))); - PCollectionTuple processingResult; - if (handler instanceof OrderedProcessingGlobalSequenceHandler) { - OrderedProcessingGlobalSequenceHandler globalSequenceHandler = (OrderedProcessingGlobalSequenceHandler) handler; - - boolean streamingProcessing = input.isBounded() == IsBounded.UNBOUNDED; - - final PCollectionView latestContinuousSequence = - input - .apply("Convert to SequenceAndTimestamp", - ParDo.of(new ToTimestampedEventConverter<>())) - .apply("Global Sequence Tracker", - streamingProcessing ? - new GlobalSequenceTracker<>( - globalSequenceHandler.getGlobalSequenceCombiner(), - globalSequenceHandler.getGlobalSequenceGenerationFrequency()) : - new GlobalSequenceTracker<>( - globalSequenceHandler.getGlobalSequenceCombiner())); - - if (streamingProcessing) { - PCollection>> tickers = input.apply("Create Tickers", - new PerKeyTickerGenerator<>(keyCoder, eventCoder, - globalSequenceHandler.getFrequencyOfCheckingForNewGlobalSequence())); - - input = - PCollectionList.of(input).and(tickers) - .apply("Combine Events and Tickers", Flatten.pCollections()) - .setCoder(tickers.getCoder()); - } - processingResult = - input - .apply( - ParDo.of( - new GlobalSequencesProcessorDoFn<>( - handler.getEventExaminer(), - eventCoder, - stateCoder, - keyCoder, - mainOutput, - statusOutput, - handler.getStatusUpdateFrequency(), - unprocessedEventOutput, - handler.isProduceStatusUpdateOnEveryEvent(), - handler.getMaxOutputElementsPerBundle(), - latestContinuousSequence) - ) - .withOutputTags( - mainOutput, - TupleTagList.of(Arrays.asList(statusOutput, unprocessedEventOutput))) - .withSideInput(GLOBAL_SEQUENCE_TRACKER, latestContinuousSequence) - ); - return new OrderedEventProcessorResult<>( - pipeline, - processingResult.get(mainOutput).setCoder(mainOutputCoder), - mainOutput, - processingResult.get(statusOutput).setCoder(processingStatusCoder), - statusOutput, - processingResult.get(unprocessedEventOutput).setCoder(unprocessedEventsCoder), - unprocessedEventOutput, - latestContinuousSequence); + OrderedProcessingGlobalSequenceHandler globalSequenceHandler = + (OrderedProcessingGlobalSequenceHandler) handler; + + return expandGlobalSequenceProcessing(input, mainOutput, statusOutput, + unprocessedEventOutput, handler, pipeline, keyCoder, eventCoder, stateCoder, + mainOutputCoder, + processingStatusCoder, unprocessedEventsCoder, globalSequenceHandler); } else { - // Per key sequence handler - processingResult = - input.apply( - ParDo.of( - new SequencePerKeyProcessorDoFn<>( - handler.getEventExaminer(), - eventCoder, - stateCoder, - keyCoder, - mainOutput, - statusOutput, - handler.getStatusUpdateFrequency(), - unprocessedEventOutput, - handler.isProduceStatusUpdateOnEveryEvent(), - handler.getMaxOutputElementsPerBundle())) - .withOutputTags( - mainOutput, - TupleTagList.of(Arrays.asList(statusOutput, unprocessedEventOutput)))); - return new OrderedEventProcessorResult<>( - pipeline, - processingResult.get(mainOutput).setCoder(mainOutputCoder), - mainOutput, - processingResult.get(statusOutput).setCoder(processingStatusCoder), - statusOutput, - processingResult.get(unprocessedEventOutput).setCoder(unprocessedEventsCoder), - unprocessedEventOutput); + return expandPerKeyProcessing(input, mainOutput, statusOutput, + unprocessedEventOutput, handler, pipeline, keyCoder, eventCoder, stateCoder, + mainOutputCoder, + processingStatusCoder, unprocessedEventsCoder); + } + } + + private OrderedEventProcessorResult expandPerKeyProcessing( + PCollection>> input, + TupleTag> mainOutput, + TupleTag> statusOutput, + TupleTag>>> unprocessedEventOutput, + OrderedProcessingHandler handler, Pipeline pipeline, + Coder keyCoder, Coder eventCoder, Coder stateCoder, + KvCoder mainOutputCoder, + KvCoder processingStatusCoder, + KvCoder>> unprocessedEventsCoder) { + PCollectionTuple processingResult; + processingResult = + input.apply( + ParDo.of( + new SequencePerKeyProcessorDoFn<>( + handler.getEventExaminer(), + eventCoder, + stateCoder, + keyCoder, + mainOutput, + statusOutput, + handler.getStatusUpdateFrequency(), + unprocessedEventOutput, + handler.isProduceStatusUpdateOnEveryEvent(), + handler.getMaxOutputElementsPerBundle())) + .withOutputTags( + mainOutput, + TupleTagList.of(Arrays.asList(statusOutput, unprocessedEventOutput)))); + return new OrderedEventProcessorResult<>( + pipeline, + processingResult.get(mainOutput).setCoder(mainOutputCoder), + mainOutput, + processingResult.get(statusOutput).setCoder(processingStatusCoder), + statusOutput, + processingResult.get(unprocessedEventOutput).setCoder(unprocessedEventsCoder), + unprocessedEventOutput); + } + + private OrderedEventProcessorResult expandGlobalSequenceProcessing( + PCollection>> input, + TupleTag> mainOutput, + TupleTag> statusOutput, + TupleTag>>> unprocessedEventOutput, + OrderedProcessingHandler handler, Pipeline pipeline, + Coder keyCoder, Coder eventCoder, Coder stateCoder, + KvCoder mainOutputCoder, + KvCoder processingStatusCoder, + KvCoder>> unprocessedEventsCoder, + OrderedProcessingGlobalSequenceHandler globalSequenceHandler) { + PCollectionTuple processingResult; + boolean streamingProcessing = input.isBounded() == IsBounded.UNBOUNDED; + + final PCollectionView latestContiguousRange = + input + .apply("Convert to SequenceAndTimestamp", + ParDo.of(new ToTimestampedEventConverter<>())) + .apply("Global Sequence Tracker", + streamingProcessing ? + new GlobalSequenceTracker<>( + globalSequenceHandler.getGlobalSequenceCombiner(), + globalSequenceHandler.getGlobalSequenceGenerationFrequency()) : + new GlobalSequenceTracker<>( + globalSequenceHandler.getGlobalSequenceCombiner())); + + if (streamingProcessing) { + PCollection>> tickers = input.apply("Create Tickers", + new PerKeyTickerGenerator<>(keyCoder, eventCoder, + globalSequenceHandler.getFrequencyOfCheckingForNewGlobalSequence())); + input = + PCollectionList.of(input).and(tickers) + .apply("Combine Events and Tickers", Flatten.pCollections()) + .setCoder(tickers.getCoder()); } + processingResult = + input + .apply( + ParDo.of( + new GlobalSequencesProcessorDoFn<>( + handler.getEventExaminer(), + eventCoder, + stateCoder, + keyCoder, + mainOutput, + statusOutput, + handler.getStatusUpdateFrequency(), + unprocessedEventOutput, + handler.isProduceStatusUpdateOnEveryEvent(), + handler.getMaxOutputElementsPerBundle(), + latestContiguousRange) + ) + .withOutputTags( + mainOutput, + TupleTagList.of(Arrays.asList(statusOutput, unprocessedEventOutput))) + .withSideInput(GLOBAL_SEQUENCE_TRACKER, latestContiguousRange) + ); + return new OrderedEventProcessorResult<>( + pipeline, + processingResult.get(mainOutput).setCoder(mainOutputCoder), + mainOutput, + processingResult.get(statusOutput).setCoder(processingStatusCoder), + statusOutput, + processingResult.get(unprocessedEventOutput).setCoder(unprocessedEventsCoder), + unprocessedEventOutput, + latestContiguousRange); } private static Coder getOrderedProcessingStatusCoder(Pipeline pipeline) { diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorResult.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorResult.java index 2d4966b3c4a0..e1ee4c7d6e50 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorResult.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorResult.java @@ -37,6 +37,9 @@ * reason *

  • processingStatuses - the key/value of the status of processing for a particular key * + * In case of global sequence processing, the result also contains PCollectionView of the latest + * contiguous sequence range + * * @param * @param */ @@ -52,7 +55,7 @@ public class OrderedEventProcessorResult implements POutp unprocessedEventPCollection; private final TupleTag>>> unprocessedEventTupleTag; - private final @Nullable PCollectionView latestCompletedSequenceRange; + private final @Nullable PCollectionView latestContiguousRange; OrderedEventProcessorResult( Pipeline pipeline, @@ -75,7 +78,7 @@ public class OrderedEventProcessorResult implements POutp TupleTag> eventProcessingStatusTupleTag, PCollection>>> unprocessedEventPCollection, TupleTag>>> unprocessedEventTupleTag, - @Nullable PCollectionView latestCompletedSequenceRange) { + @Nullable PCollectionView latestContiguousRange) { this.pipeline = pipeline; this.outputPCollection = outputPCollection; @@ -84,7 +87,7 @@ public class OrderedEventProcessorResult implements POutp this.eventProcessingStatusTupleTag = eventProcessingStatusTupleTag; this.unprocessedEventPCollection = unprocessedEventPCollection; this.unprocessedEventTupleTag = unprocessedEventTupleTag; - this.latestCompletedSequenceRange = latestCompletedSequenceRange; + this.latestContiguousRange = latestContiguousRange; } private final Pipeline pipeline; @@ -129,7 +132,7 @@ public PCollection>>> unprocessedEven return unprocessedEventPCollection; } - public @Nullable PCollectionView latestCompletedSequenceRange() { - return latestCompletedSequenceRange; + public @Nullable PCollectionView latestContiguousRange() { + return latestContiguousRange; } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java index 3ff4416c57bc..9587d26b5f96 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java @@ -34,6 +34,11 @@ /** * Parent class for Ordered Processing configuration handlers. * + * There are two types of processing - when the sequence numbers are contiguous per key and these + * sequences per keys are independent of each other, and when there is a global sequence shared + * by all keys. In case of the global sequence processing the custom handler must extend from + * {@see OrderedProcessingGlobalSequenceHandler}. + * * @param type of events to be processed * @param type of keys which will be used to group the events * @param type of internal State which will be used for processing @@ -224,6 +229,15 @@ public void setMaxOutputElementsPerBundle(int maxOutputElementsPerBundle) { this.maxOutputElementsPerBundle = maxOutputElementsPerBundle; } + /** + * Parent class for Ordered Processing configuration handlers to handle processing of the events + * where global sequence is used. + * + * @param type of events to be processed + * @param type of keys which will be used to group the events + * @param type of internal State which will be used for processing + * @param type of the result of the processing which will be output + */ public abstract static class OrderedProcessingGlobalSequenceHandler< EventT, KeyT, StateT extends MutableState, ResultT> extends OrderedProcessingHandler { @@ -236,11 +250,26 @@ public OrderedProcessingGlobalSequenceHandler( super(eventTClass, keyTClass, stateTClass, resultTClass); } + /** + * Provide the global sequence combiner. Default is to use {@link DefaultSequenceCombiner}. + * + * @return combiner + */ public GloballyAsSingletonView>>, ContiguousSequenceRange> getGlobalSequenceCombiner() { return Combine.globally( new DefaultSequenceCombiner(getEventExaminer())).asSingletonView(); } + /** + * How frequently the combiner should generate a new sequence? This parameter only affects the + * behaviour of streaming pipelines. + *

    + * Notice that some runners cache the output of side inputs and this parameter might not appear + * to have an effect unless the cache time-to-live is equal or less than this frequency. For + * Dataflow runner, see {@link this Dataflow streaming pipeline option} + * + * @return frequency of generating new global sequence. Default - every second. + */ public Duration getFrequencyOfCheckingForNewGlobalSequence() { return Duration.standardSeconds(1); } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/PerKeyTickerGenerator.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/PerKeyTickerGenerator.java index 24bd4465e2a3..4856f69e0725 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/PerKeyTickerGenerator.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/PerKeyTickerGenerator.java @@ -24,6 +24,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * PTransform to generate per key tickers with certain frequency. + * + * @param + * @param + */ class PerKeyTickerGenerator extends PTransform>>, PCollection>>> { @@ -100,7 +106,7 @@ public void onTimer( return; } - // null value will be an indicator to the main transform that the element is a ticker + // Null value will be an indicator to the main transform that the element is a ticker outputReceiver.output(KV.of(key, KV.of(0L, null))); tickerTimer.offset(tickerFrequency).setRelative(); } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java index cb23d680a774..b3035aebdc9d 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java @@ -19,6 +19,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * Stateful DoFn to process per key sequences. + * + * @param event type + * @param event key type + * @param result type + * @param state type + */ class SequencePerKeyProcessorDoFn> extends ProcessorDoFn { diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/package-info.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/package-info.java index f9d7e3d67bff..8506150e0f66 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/package-info.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/package-info.java @@ -16,7 +16,8 @@ * limitations under the License. */ /** - * Provides a transform for ordered processing. + * Provides a transform for ordered processing. For a detailed reference implementation which uses + * this transform visit {@link https://github.com/GoogleCloudPlatform/dataflow-ordered-processing} * * @see org.apache.beam.sdk.extensions.ordered.OrderedEventProcessor */ diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTestBase.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTestBase.java index 1dd49edd4f1e..79f874ce5961 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTestBase.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTestBase.java @@ -206,10 +206,10 @@ protected void doTest( } if (expectedLastCompletedSequence != null - && processingResult.latestCompletedSequenceRange() != null) { + && processingResult.latestContiguousRange() != null) { PCollection globalSequences = rawInput .apply("Publish Global Sequences", - new GlobalSequenceRangePublisher(processingResult.latestCompletedSequenceRange(), + new GlobalSequenceRangePublisher(processingResult.latestContiguousRange(), handler.getKeyCoder(pipeline, input.getCoder()), handler.getEventCoder(pipeline, input.getCoder()))); PAssert.that("CompletedSequenceRange verification", globalSequences).satisfies( From 1e39b9877ee7c2406c5f7eabdcde84cb4d61e5f8 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Mon, 23 Sep 2024 15:09:39 -0700 Subject: [PATCH 19/33] Fixed the description in CHANGES.md --- CHANGES.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 0d2e2d3a7b50..cb97c546da65 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -68,7 +68,7 @@ * Dataflow worker can install packages from Google Artifact Registry Python repositories (Python) ([#32123](https://github.com/apache/beam/issues/32123)). * Added support for Zstd codec in SerializableAvroCodecFactory (Java) ([#32349](https://github.com/apache/beam/issues/32349)) -* Added support for processing events which use a global sequence to OrderedEventProcessor (Java) +* Added support for processing events which use a global sequence to "ordered" extension (Java) * X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). ## Breaking Changes From 13c4d90feaa59180fac00890faa249d1bf9704d9 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Mon, 23 Sep 2024 15:33:03 -0700 Subject: [PATCH 20/33] Polish by "spotless" --- .../ordered/ContiguousSequenceRange.java | 50 ++- .../sdk/extensions/ordered/EventExaminer.java | 6 +- .../ordered/GlobalSequenceTracker.java | 65 ++- .../ordered/GlobalSequencesProcessorDoFn.java | 81 ++-- .../ordered/OrderedEventProcessor.java | 171 ++++---- .../ordered/OrderedEventProcessorResult.java | 26 +- .../ordered/OrderedProcessingHandler.java | 62 +-- .../ordered/OrderedProcessingStatus.java | 37 +- .../ordered/PerKeyTickerGenerator.java | 38 +- .../extensions/ordered/ProcessingState.java | 69 ++-- .../sdk/extensions/ordered/ProcessorDoFn.java | 80 ++-- .../ordered/SequencePerKeyProcessorDoFn.java | 67 +++- .../combiner/DefaultSequenceCombiner.java | 63 ++- .../combiner/SequenceRangeAccumulator.java | 148 ++++--- .../sdk/extensions/ordered/package-info.java | 3 +- ...deredEventProcessorGlobalSequenceTest.java | 179 +++++---- ...deredEventProcessorPerKeySequenceTest.java | 121 +++--- .../OrderedEventProcessorTestBase.java | 109 ++--- .../StringBufferOrderedProcessingHandler.java | 4 +- .../SequenceRangeAccumulatorCoderTest.java | 21 +- .../SequenceRangeAccumulatorTest.java | 379 ++++++++++-------- 21 files changed, 1050 insertions(+), 729 deletions(-) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ContiguousSequenceRange.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ContiguousSequenceRange.java index b36f4ffa1ea4..7bfebcb6a0de 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ContiguousSequenceRange.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ContiguousSequenceRange.java @@ -1,3 +1,20 @@ +/* + * 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.extensions.ordered; import com.google.auto.value.AutoValue; @@ -13,28 +30,20 @@ import org.checkerframework.checker.nullness.qual.UnknownKeyFor; import org.joda.time.Instant; -/** - * A range of contiguous event sequences and the latest timestamp of the events in the range. - */ +/** A range of contiguous event sequences and the latest timestamp of the events in the range. */ @AutoValue public abstract class ContiguousSequenceRange { public static final ContiguousSequenceRange EMPTY = - ContiguousSequenceRange.of(Long.MIN_VALUE, Long.MIN_VALUE, Instant.ofEpochMilli( - Long.MIN_VALUE)); + ContiguousSequenceRange.of( + Long.MIN_VALUE, Long.MIN_VALUE, Instant.ofEpochMilli(Long.MIN_VALUE)); - /** - * @return inclusive starting sequence - */ + /** @return inclusive starting sequence */ public abstract long getStart(); - /** - * @return inclusive end sequence - */ + /** @return inclusive end sequence */ public abstract long getEnd(); - /** - * @return latest timestamp of all events in the range - */ + /** @return latest timestamp of all events in the range */ public abstract Instant getTimestamp(); public static ContiguousSequenceRange of(long start, long end, Instant timestamp) { @@ -49,13 +58,13 @@ static CompletedSequenceRangeCoder of() { return INSTANCE; } - private CompletedSequenceRangeCoder() { - } + private CompletedSequenceRangeCoder() {} @Override - public void encode(ContiguousSequenceRange value, - @UnknownKeyFor @NonNull @Initialized OutputStream outStream) - throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { + public void encode( + ContiguousSequenceRange value, @UnknownKeyFor @NonNull @Initialized OutputStream outStream) + throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull + @Initialized IOException { VarLongCoder.of().encode(value.getStart(), outStream); VarLongCoder.of().encode(value.getEnd(), outStream); InstantCoder.of().encode(value.getTimestamp(), outStream); @@ -63,7 +72,8 @@ public void encode(ContiguousSequenceRange value, @Override public ContiguousSequenceRange decode(@UnknownKeyFor @NonNull @Initialized InputStream inStream) - throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { + throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull + @Initialized IOException { long start = VarLongCoder.of().decode(inStream); long end = VarLongCoder.of().decode(inStream); Instant timestamp = InstantCoder.of().decode(inStream); diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/EventExaminer.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/EventExaminer.java index 15f49546d323..b5de67f16ced 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/EventExaminer.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/EventExaminer.java @@ -31,8 +31,8 @@ public interface EventExaminer> extends Serializable { /** - * Is this event the first expected event for the given key and window if the per key sequence - * is used? In case of global sequence it determines the first global sequence event. + * Is this event the first expected event for the given key and window if the per key sequence is + * used? In case of global sequence it determines the first global sequence event. * * @param sequenceNumber the sequence number of the event as defined by the key of the input * PCollection to {@link OrderedEventProcessor} @@ -54,7 +54,7 @@ public interface EventExaminer> /** * Is this event the last expected event for a given key and window? * - * Note, this method is not used yet with global sequences. + *

    Note, this method is not used yet with global sequences. * * @param sequenceNumber of the event * @param event being processed diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java index 05b6d9d09f89..671ca5dd4873 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java @@ -1,3 +1,20 @@ +/* + * 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.extensions.ordered; import org.apache.beam.sdk.extensions.ordered.ContiguousSequenceRange.CompletedSequenceRangeCoder; @@ -20,24 +37,33 @@ * PTransform to produce the side input of the maximum contiguous range of sequence numbers * * @param type of event key - * @param type of event - * @param type of processing result - * @param type of state + * @param type of event + * @param type of processing result + * @param type of state */ -class GlobalSequenceTracker> extends - PTransform>>>, PCollectionView> { +class GlobalSequenceTracker< + EventKeyT, EventT, ResultT, StateT extends MutableState> + extends PTransform< + PCollection>>>, + PCollectionView> { - private final Combine.GloballyAsSingletonView>>, ContiguousSequenceRange> sideInputProducer; + private final Combine.GloballyAsSingletonView< + TimestampedValue>>, ContiguousSequenceRange> + sideInputProducer; private final @Nullable Duration frequencyOfGeneration; public GlobalSequenceTracker( - Combine.GloballyAsSingletonView>>, ContiguousSequenceRange> sideInputProducer) { + Combine.GloballyAsSingletonView< + TimestampedValue>>, ContiguousSequenceRange> + sideInputProducer) { this.sideInputProducer = sideInputProducer; this.frequencyOfGeneration = null; } public GlobalSequenceTracker( - Combine.GloballyAsSingletonView>>, ContiguousSequenceRange> sideInputProducer, + Combine.GloballyAsSingletonView< + TimestampedValue>>, ContiguousSequenceRange> + sideInputProducer, Duration globalSequenceGenerationFrequency) { this.sideInputProducer = sideInputProducer; this.frequencyOfGeneration = globalSequenceGenerationFrequency; @@ -46,28 +72,31 @@ public GlobalSequenceTracker( @Override public PCollectionView expand( PCollection>>> input) { - input.getPipeline().getCoderRegistry().registerCoderForClass( - ContiguousSequenceRange.class, CompletedSequenceRangeCoder.of()); + input + .getPipeline() + .getCoderRegistry() + .registerCoderForClass(ContiguousSequenceRange.class, CompletedSequenceRangeCoder.of()); if (frequencyOfGeneration != null) { // This branch will only be executed in case of streaming pipelines. // For batch pipelines the side input should only be computed once. - input = input - .apply("Triggering Setup", + input = + input.apply( + "Triggering Setup", // Reproduce the windowing of the input PCollection, but change the triggering // in order to create a slowing changing side input Window.>>>into( (WindowFn>>, ?>) - input.getWindowingStrategy().getWindowFn() - ) + input.getWindowingStrategy().getWindowFn()) .accumulatingFiredPanes() // TODO: verify that we don't need to have the lateness parameterized .withAllowedLateness(Duration.ZERO) .triggering( - Repeatedly.forever(AfterFirst.of( - AfterPane.elementCountAtLeast(1), - AfterProcessingTime.pastFirstElementInPane() - .plusDelayOf(frequencyOfGeneration))))); + Repeatedly.forever( + AfterFirst.of( + AfterPane.elementCountAtLeast(1), + AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(frequencyOfGeneration))))); } return input.apply("Create Side Input", sideInputProducer); } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java index 8226e1ff5046..a3d84c6109a9 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java @@ -1,3 +1,20 @@ +/* + * 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.extensions.ordered; import org.apache.beam.sdk.coders.BooleanCoder; @@ -28,8 +45,8 @@ * @param * @param */ -class GlobalSequencesProcessorDoFn> +class GlobalSequencesProcessorDoFn< + EventT, EventKeyT, ResultT, StateT extends MutableState> extends ProcessorDoFn { private static final Logger LOG = LoggerFactory.getLogger(GlobalSequencesProcessorDoFn.class); @@ -41,6 +58,7 @@ class GlobalSequencesProcessorDoFn> bufferedEventsSpec; @@ -63,19 +81,25 @@ class GlobalSequencesProcessorDoFn latestContiguousRangeSideInput; - GlobalSequencesProcessorDoFn(EventExaminer eventExaminer, + GlobalSequencesProcessorDoFn( + EventExaminer eventExaminer, Coder eventCoder, Coder stateCoder, Coder keyCoder, TupleTag> mainOutputTupleTag, TupleTag> statusTupleTag, Duration statusUpdateFrequency, - TupleTag>>> - unprocessedEventTupleTag, - boolean produceStatusUpdateOnEveryEvent, long maxNumberOfResultsToProduce, + TupleTag>>> unprocessedEventTupleTag, + boolean produceStatusUpdateOnEveryEvent, + long maxNumberOfResultsToProduce, PCollectionView latestContiguousRangeSideInput) { - super(eventExaminer, mainOutputTupleTag, statusTupleTag, - statusUpdateFrequency, unprocessedEventTupleTag, produceStatusUpdateOnEveryEvent, + super( + eventExaminer, + mainOutputTupleTag, + statusTupleTag, + statusUpdateFrequency, + unprocessedEventTupleTag, + produceStatusUpdateOnEveryEvent, maxNumberOfResultsToProduce); this.latestContiguousRangeSideInput = latestContiguousRangeSideInput; @@ -96,19 +120,19 @@ boolean checkForSequenceGapInBufferedEvents() { } @ProcessElement - public void processElement(ProcessContext context, + public void processElement( + ProcessContext context, @Element KV> eventAndSequence, @StateId(BUFFERED_EVENTS) OrderedListState bufferedEventsProxy, @AlwaysFetched @StateId(PROCESSING_STATE) - ValueState> processingStateProxy, + ValueState> processingStateProxy, @StateId(MUTABLE_STATE) ValueState mutableStateProxy, @TimerId(STATUS_EMISSION_TIMER) Timer statusEmissionTimer, @TimerId(BATCH_EMISSION_TIMER) Timer batchEmissionTimer, MultiOutputReceiver outputReceiver, BoundedWindow window) { - ContiguousSequenceRange lastContiguousRange = context.sideInput( - latestContiguousRangeSideInput); + ContiguousSequenceRange lastContiguousRange = context.sideInput(latestContiguousRangeSideInput); EventT event = eventAndSequence.getValue().getValue(); EventKeyT key = eventAndSequence.getKey(); @@ -132,7 +156,8 @@ public void processElement(ProcessContext context, processingState.updateGlobalSequenceDetails(lastContiguousRange); if (event == null) { - // This is a ticker event. We only need to update the state as it relates to the global sequence. + // This is a ticker event. We only need to update the state as it relates to the global + // sequence. processingStateProxy.write(processingState); setBatchEmissionTimerIfNeeded(batchEmissionTimer, processingState); @@ -167,11 +192,11 @@ public void processElement(ProcessContext context, setBatchEmissionTimerIfNeeded(batchEmissionTimer, processingState); } - private void setBatchEmissionTimerIfNeeded(Timer batchEmissionTimer, - ProcessingState processingState) { + private void setBatchEmissionTimerIfNeeded( + Timer batchEmissionTimer, ProcessingState processingState) { ContiguousSequenceRange lastCompleteGlobalSequence = processingState.getLastContiguousRange(); - if (lastCompleteGlobalSequence != null && - processingState.thereAreGloballySequencedEventsToBeProcessed()) { + if (lastCompleteGlobalSequence != null + && processingState.thereAreGloballySequencedEventsToBeProcessed()) { batchEmissionTimer.set(lastCompleteGlobalSequence.getTimestamp()); } } @@ -181,12 +206,13 @@ public void onBatchEmission( OnTimerContext context, @StateId(BUFFERED_EVENTS) OrderedListState bufferedEventsState, @AlwaysFetched @StateId(PROCESSING_STATE) - ValueState> processingStatusState, + ValueState> processingStatusState, @AlwaysFetched @StateId(MUTABLE_STATE) ValueState mutableStateState, @TimerId(BATCH_EMISSION_TIMER) Timer batchEmissionTimer, MultiOutputReceiver outputReceiver) { - // At this point everything in the buffered state is ready to be processed up to the latest global sequence. + // At this point everything in the buffered state is ready to be processed up to the latest + // global sequence. @Nullable ProcessingState processingState = processingStatusState.read(); if (processingState == null) { LOG.warn("Missing the processing state. Probably occurred during pipeline drainage"); @@ -213,8 +239,14 @@ public void onBatchEmission( this.numberOfResultsBeforeBundleStart = processingState.getResultCount(); - state = processBufferedEventRange(processingState, state, bufferedEventsState, outputReceiver, - batchEmissionTimer, lastContiguousRange); + state = + processBufferedEventRange( + processingState, + state, + bufferedEventsState, + outputReceiver, + batchEmissionTimer, + lastContiguousRange); saveStates( processingStatusState, @@ -232,10 +264,9 @@ public void onStatusEmission( MultiOutputReceiver outputReceiver, @TimerId(STATUS_EMISSION_TIMER) Timer statusEmissionTimer, @StateId(WINDOW_CLOSED) ValueState windowClosedState, - @StateId(PROCESSING_STATE) - ValueState> processingStateState) { + @StateId(PROCESSING_STATE) ValueState> processingStateState) { - processStatusTimerEvent(outputReceiver, statusEmissionTimer, windowClosedState, - processingStateState); + processStatusTimerEvent( + outputReceiver, statusEmissionTimer, windowClosedState, processingStateState); } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java index 96b0db50c019..913c3be7aea2 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java @@ -50,41 +50,41 @@ * Transform for processing ordered events. Events are grouped by the key and within each key they * are applied according to the provided sequence. Events which arrive out of sequence are buffered * and processed after all the missing events for a given key have arrived. - *

    - * There are two sequencing modes - a sequence per key and a global sequence. See - * {@link OrderedProcessingHandler} for details on how to configure this transform. * - * @param type of event + *

    There are two sequencing modes - a sequence per key and a global sequence. See {@link + * OrderedProcessingHandler} for details on how to configure this transform. + * + * @param type of event * @param type of event key - * @param type of the state + * @param type of the state */ @AutoValue @SuppressWarnings({"nullness", "TypeNameShadowing"}) public abstract class OrderedEventProcessor< - EventT, EventKeyT, ResultT, StateT extends MutableState> + EventT, EventKeyT, ResultT, StateT extends MutableState> extends PTransform< - PCollection>>, - OrderedEventProcessorResult> { + PCollection>>, + OrderedEventProcessorResult> { public static final String GLOBAL_SEQUENCE_TRACKER = "global_sequence_tracker"; /** * Create the transform. * - * @param handler provides the configuration of this transform - * @param type of event + * @param handler provides the configuration of this transform + * @param type of event * @param type of event key - * @param type of the result object - * @param type of the state to store + * @param type of the result object + * @param type of the state to store * @return the transform */ public static < - EventTypeT, - EventKeyTypeT, - ResultTypeT, - StateTypeT extends MutableState> - OrderedEventProcessor create( - OrderedProcessingHandler handler) { + EventTypeT, + EventKeyTypeT, + ResultTypeT, + StateTypeT extends MutableState> + OrderedEventProcessor create( + OrderedProcessingHandler handler) { return new AutoValue_OrderedEventProcessor<>(handler); } @@ -95,15 +95,12 @@ OrderedEventProcessor create public OrderedEventProcessorResult expand( PCollection>> input) { final TupleTag> mainOutput = - new TupleTag>("mainOutput") { - }; + new TupleTag>("mainOutput") {}; final TupleTag> statusOutput = - new TupleTag>("status") { - }; + new TupleTag>("status") {}; final TupleTag>>> unprocessedEventOutput = - new TupleTag>>>("unprocessed-events") { - }; + new TupleTag>>>("unprocessed-events") {}; OrderedProcessingHandler handler = getHandler(); Pipeline pipeline = input.getPipeline(); @@ -144,18 +141,38 @@ public OrderedEventProcessorResult expand( keyCoder, KvCoder.of(VarLongCoder.of(), new UnprocessedEventCoder<>(eventCoder))); if (handler instanceof OrderedProcessingGlobalSequenceHandler) { - OrderedProcessingGlobalSequenceHandler globalSequenceHandler = - (OrderedProcessingGlobalSequenceHandler) handler; + OrderedProcessingGlobalSequenceHandler + globalSequenceHandler = + (OrderedProcessingGlobalSequenceHandler) handler; - return expandGlobalSequenceProcessing(input, mainOutput, statusOutput, - unprocessedEventOutput, handler, pipeline, keyCoder, eventCoder, stateCoder, + return expandGlobalSequenceProcessing( + input, + mainOutput, + statusOutput, + unprocessedEventOutput, + handler, + pipeline, + keyCoder, + eventCoder, + stateCoder, mainOutputCoder, - processingStatusCoder, unprocessedEventsCoder, globalSequenceHandler); + processingStatusCoder, + unprocessedEventsCoder, + globalSequenceHandler); } else { - return expandPerKeyProcessing(input, mainOutput, statusOutput, - unprocessedEventOutput, handler, pipeline, keyCoder, eventCoder, stateCoder, + return expandPerKeyProcessing( + input, + mainOutput, + statusOutput, + unprocessedEventOutput, + handler, + pipeline, + keyCoder, + eventCoder, + stateCoder, mainOutputCoder, - processingStatusCoder, unprocessedEventsCoder); + processingStatusCoder, + unprocessedEventsCoder); } } @@ -164,8 +181,11 @@ private OrderedEventProcessorResult expandPerKeyProc TupleTag> mainOutput, TupleTag> statusOutput, TupleTag>>> unprocessedEventOutput, - OrderedProcessingHandler handler, Pipeline pipeline, - Coder keyCoder, Coder eventCoder, Coder stateCoder, + OrderedProcessingHandler handler, + Pipeline pipeline, + Coder keyCoder, + Coder eventCoder, + Coder stateCoder, KvCoder mainOutputCoder, KvCoder processingStatusCoder, KvCoder>> unprocessedEventsCoder) { @@ -202,59 +222,65 @@ private OrderedEventProcessorResult expandGlobalSequ TupleTag> mainOutput, TupleTag> statusOutput, TupleTag>>> unprocessedEventOutput, - OrderedProcessingHandler handler, Pipeline pipeline, - Coder keyCoder, Coder eventCoder, Coder stateCoder, + OrderedProcessingHandler handler, + Pipeline pipeline, + Coder keyCoder, + Coder eventCoder, + Coder stateCoder, KvCoder mainOutputCoder, KvCoder processingStatusCoder, KvCoder>> unprocessedEventsCoder, - OrderedProcessingGlobalSequenceHandler globalSequenceHandler) { + OrderedProcessingGlobalSequenceHandler + globalSequenceHandler) { PCollectionTuple processingResult; boolean streamingProcessing = input.isBounded() == IsBounded.UNBOUNDED; final PCollectionView latestContiguousRange = input - .apply("Convert to SequenceAndTimestamp", - ParDo.of(new ToTimestampedEventConverter<>())) - .apply("Global Sequence Tracker", - streamingProcessing ? - new GlobalSequenceTracker<>( + .apply("Convert to SequenceAndTimestamp", ParDo.of(new ToTimestampedEventConverter<>())) + .apply( + "Global Sequence Tracker", + streamingProcessing + ? new GlobalSequenceTracker<>( globalSequenceHandler.getGlobalSequenceCombiner(), - globalSequenceHandler.getGlobalSequenceGenerationFrequency()) : - new GlobalSequenceTracker<>( + globalSequenceHandler.getGlobalSequenceGenerationFrequency()) + : new GlobalSequenceTracker<>( globalSequenceHandler.getGlobalSequenceCombiner())); if (streamingProcessing) { - PCollection>> tickers = input.apply("Create Tickers", - new PerKeyTickerGenerator<>(keyCoder, eventCoder, - globalSequenceHandler.getFrequencyOfCheckingForNewGlobalSequence())); + PCollection>> tickers = + input.apply( + "Create Tickers", + new PerKeyTickerGenerator<>( + keyCoder, + eventCoder, + globalSequenceHandler.getFrequencyOfCheckingForNewGlobalSequence())); input = - PCollectionList.of(input).and(tickers) + PCollectionList.of(input) + .and(tickers) .apply("Combine Events and Tickers", Flatten.pCollections()) .setCoder(tickers.getCoder()); } processingResult = - input - .apply( - ParDo.of( - new GlobalSequencesProcessorDoFn<>( - handler.getEventExaminer(), - eventCoder, - stateCoder, - keyCoder, - mainOutput, - statusOutput, - handler.getStatusUpdateFrequency(), - unprocessedEventOutput, - handler.isProduceStatusUpdateOnEveryEvent(), - handler.getMaxOutputElementsPerBundle(), - latestContiguousRange) - ) - .withOutputTags( + input.apply( + ParDo.of( + new GlobalSequencesProcessorDoFn<>( + handler.getEventExaminer(), + eventCoder, + stateCoder, + keyCoder, mainOutput, - TupleTagList.of(Arrays.asList(statusOutput, unprocessedEventOutput))) - .withSideInput(GLOBAL_SEQUENCE_TRACKER, latestContiguousRange) - ); + statusOutput, + handler.getStatusUpdateFrequency(), + unprocessedEventOutput, + handler.isProduceStatusUpdateOnEveryEvent(), + handler.getMaxOutputElementsPerBundle(), + latestContiguousRange)) + .withOutputTags( + mainOutput, + TupleTagList.of(Arrays.asList(statusOutput, unprocessedEventOutput))) + .withSideInput(GLOBAL_SEQUENCE_TRACKER, latestContiguousRange)); return new OrderedEventProcessorResult<>( pipeline, processingResult.get(mainOutput).setCoder(mainOutputCoder), @@ -282,15 +308,16 @@ private static Coder getOrderedProcessingStatusCoder(Pi return result; } - static class ToTimestampedEventConverter extends - DoFn>, TimestampedValue>>> { + static class ToTimestampedEventConverter + extends DoFn< + KV>, TimestampedValue>>> { @ProcessElement - public void convert(@Element KV> element, + public void convert( + @Element KV> element, @Timestamp Instant timestamp, OutputReceiver>>> outputReceiver) { outputReceiver.output(TimestampedValue.of(element, timestamp)); } } - } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorResult.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorResult.java index e1ee4c7d6e50..48b9fafc99af 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorResult.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorResult.java @@ -34,11 +34,11 @@ * The result of the ordered processing. Three PCollections are returned: *

  • output - the key/value of the mutated states *
  • unprocessedEvents - the key/value of the events that failed to be processed and the failure - * reason + * reason *
  • processingStatuses - the key/value of the status of processing for a particular key * - * In case of global sequence processing, the result also contains PCollectionView of the latest - * contiguous sequence range + *

    In case of global sequence processing, the result also contains PCollectionView of the + * latest contiguous sequence range * * @param * @param @@ -66,8 +66,15 @@ public class OrderedEventProcessorResult implements POutp PCollection>>> unprocessedEventPCollection, TupleTag>>> unprocessedEventTupleTag) { - this(pipeline, outputPCollection, outputPCollectionTupleTag, eventProcessingStatusPCollection, - eventProcessingStatusTupleTag, unprocessedEventPCollection, unprocessedEventTupleTag, null); + this( + pipeline, + outputPCollection, + outputPCollectionTupleTag, + eventProcessingStatusPCollection, + eventProcessingStatusTupleTag, + unprocessedEventPCollection, + unprocessedEventTupleTag, + null); } OrderedEventProcessorResult( @@ -110,20 +117,17 @@ public Map, PValue> expand() { @Override public void finishSpecifyingOutput( - String transformName, PInput input, PTransform transform) { - } + String transformName, PInput input, PTransform transform) {} /** * @return processing status for a particular key. The elements will have the timestamp of the - * instant the status was emitted. + * instant the status was emitted. */ public PCollection> processingStatuses() { return eventProcessingStatusPCollection; } - /** - * @return processed states keyed by the original key - */ + /** @return processed states keyed by the original key */ public PCollection> output() { return outputPCollection; } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java index 9587d26b5f96..4844f12bd961 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java @@ -34,18 +34,18 @@ /** * Parent class for Ordered Processing configuration handlers. * - * There are two types of processing - when the sequence numbers are contiguous per key and these - * sequences per keys are independent of each other, and when there is a global sequence shared - * by all keys. In case of the global sequence processing the custom handler must extend from - * {@see OrderedProcessingGlobalSequenceHandler}. + *

    There are two types of processing - when the sequence numbers are contiguous per key and these + * sequences per keys are independent of each other, and when there is a global sequence shared by + * all keys. In case of the global sequence processing the custom handler must extend from {@see + * OrderedProcessingGlobalSequenceHandler}. * - * @param type of events to be processed - * @param type of keys which will be used to group the events - * @param type of internal State which will be used for processing + * @param type of events to be processed + * @param type of keys which will be used to group the events + * @param type of internal State which will be used for processing * @param type of the result of the processing which will be output */ public abstract class OrderedProcessingHandler< - EventT, KeyT, StateT extends MutableState, ResultT> + EventT, KeyT, StateT extends MutableState, ResultT> implements Serializable { private static final int DEFAULT_STATUS_UPDATE_FREQUENCY_SECONDS = 5; @@ -65,9 +65,9 @@ public abstract class OrderedProcessingHandler< /** * Provide concrete classes which will be used by the ordered processing transform. * - * @param eventTClass class of the events - * @param keyTClass class of the keys - * @param stateTClass class of the state + * @param eventTClass class of the events + * @param keyTClass class of the keys + * @param stateTClass class of the state * @param resultTClass class of the results */ public OrderedProcessingHandler( @@ -81,9 +81,7 @@ public OrderedProcessingHandler( this.resultTClass = resultTClass; } - /** - * @return the event examiner instance which will be used by the transform. - */ + /** @return the event examiner instance which will be used by the transform. */ public abstract @NonNull EventExaminer getEventExaminer(); /** @@ -93,11 +91,11 @@ public OrderedProcessingHandler( * PCollection. If the input PCollection doesn't use KVCoder, it will attempt to get the coder * from the pipeline's coder registry. * - * @param pipeline of the transform + * @param pipeline of the transform * @param inputCoder input coder of the transform * @return event coder * @throws CannotProvideCoderException if the method can't determine the coder based on the above - * algorithm. + * algorithm. */ public @NonNull Coder getEventCoder( Pipeline pipeline, Coder>> inputCoder) @@ -137,7 +135,7 @@ public OrderedProcessingHandler( * @param inputCoder * @return * @throws CannotProvideCoderException if the method can't determine the coder based on the above - * algorithm. + * algorithm. */ public @NonNull Coder getKeyCoder( Pipeline pipeline, Coder>> inputCoder) @@ -168,7 +166,7 @@ public OrderedProcessingHandler( *

    Default is 5 seconds. * * @return the frequency of updates. If null is returned, no updates will be emitted on a - * scheduled basis. + * scheduled basis. */ public @Nullable Duration getStatusUpdateFrequency() { return statusUpdateFrequency; @@ -233,14 +231,14 @@ public void setMaxOutputElementsPerBundle(int maxOutputElementsPerBundle) { * Parent class for Ordered Processing configuration handlers to handle processing of the events * where global sequence is used. * - * @param type of events to be processed - * @param type of keys which will be used to group the events - * @param type of internal State which will be used for processing + * @param type of events to be processed + * @param type of keys which will be used to group the events + * @param type of internal State which will be used for processing * @param type of the result of the processing which will be output */ public abstract static class OrderedProcessingGlobalSequenceHandler< - EventT, KeyT, StateT extends MutableState, ResultT> extends - OrderedProcessingHandler { + EventT, KeyT, StateT extends MutableState, ResultT> + extends OrderedProcessingHandler { public OrderedProcessingGlobalSequenceHandler( Class eventTClass, @@ -255,18 +253,22 @@ public OrderedProcessingGlobalSequenceHandler( * * @return combiner */ - public GloballyAsSingletonView>>, ContiguousSequenceRange> getGlobalSequenceCombiner() { - return Combine.globally( - new DefaultSequenceCombiner(getEventExaminer())).asSingletonView(); + public GloballyAsSingletonView< + TimestampedValue>>, ContiguousSequenceRange> + getGlobalSequenceCombiner() { + return Combine.globally(new DefaultSequenceCombiner(getEventExaminer())) + .asSingletonView(); } /** * How frequently the combiner should generate a new sequence? This parameter only affects the * behaviour of streaming pipelines. - *

    - * Notice that some runners cache the output of side inputs and this parameter might not appear - * to have an effect unless the cache time-to-live is equal or less than this frequency. For - * Dataflow runner, see {@link this Dataflow streaming pipeline option} + * + *

    Notice that some runners cache the output of side inputs and this parameter might not + * appear to have an effect unless the cache time-to-live is equal or less than this frequency. + * For Dataflow runner, see {@link this + * Dataflow streaming pipeline option} * * @return frequency of generating new global sequence. Default - every second. */ diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingStatus.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingStatus.java index de57ef46e1b9..7a556de1017b 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingStatus.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingStatus.java @@ -24,9 +24,7 @@ import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.joda.time.Instant; -/** - * Indicates the status of ordered processing for a particular key. - */ +/** Indicates the status of ordered processing for a particular key. */ @AutoValue @DefaultSchema(AutoValueSchema.class) public abstract class OrderedProcessingStatus { @@ -55,49 +53,38 @@ public static OrderedProcessingStatus create( /** * @return Last sequence processed. If null is returned - no elements for the given key and window - * have been processed yet. + * have been processed yet. */ - public abstract @Nullable Long getLastProcessedSequence(); - /** - * @return Number of events received out of sequence and buffered. - */ + /** @return Number of events received out of sequence and buffered. */ public abstract long getNumberOfBufferedEvents(); - /** - * @return Earliest buffered sequence. If null is returned - there are no buffered events. - */ + /** @return Earliest buffered sequence. If null is returned - there are no buffered events. */ @Nullable public abstract Long getEarliestBufferedSequence(); - /** - * @return Latest buffered sequence. If null is returned - there are no buffered events. - */ + /** @return Latest buffered sequence. If null is returned - there are no buffered events. */ @Nullable public abstract Long getLatestBufferedSequence(); - /** - * @return Total number of events received for the given key and window. - */ + /** @return Total number of events received for the given key and window. */ public abstract long getNumberOfReceivedEvents(); /** - * @return Number of duplicate events which were output in - * {@link OrderedEventProcessorResult#unprocessedEvents()} PCollection + * @return Number of duplicate events which were output in {@link + * OrderedEventProcessorResult#unprocessedEvents()} PCollection */ public abstract long getDuplicateCount(); - /** - * @return Number of output results produced. - */ + /** @return Number of output results produced. */ public abstract long getResultCount(); /** * @return Indicator that the last event for the given key and window has been received. It - * doesn't necessarily mean that all the events for the given key and window have been processed. - * Use {@link OrderedProcessingStatus#getNumberOfBufferedEvents()} == 0 and this indicator as the - * sign that the processing is complete. + * doesn't necessarily mean that all the events for the given key and window have been + * processed. Use {@link OrderedProcessingStatus#getNumberOfBufferedEvents()} == 0 and this + * indicator as the sign that the processing is complete. */ public abstract boolean isLastEventReceived(); diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/PerKeyTickerGenerator.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/PerKeyTickerGenerator.java index 4856f69e0725..a18ba53f5266 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/PerKeyTickerGenerator.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/PerKeyTickerGenerator.java @@ -1,3 +1,20 @@ +/* + * 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.extensions.ordered; import org.apache.beam.sdk.coders.Coder; @@ -31,8 +48,9 @@ * @param */ class PerKeyTickerGenerator - extends PTransform>>, - PCollection>>> { + extends PTransform< + PCollection>>, + PCollection>>> { private static final Logger LOG = LoggerFactory.getLogger(PerKeyTickerGenerator.class); @@ -40,8 +58,8 @@ class PerKeyTickerGenerator private final Coder eventCoder; private final Duration tickerFrequency; - PerKeyTickerGenerator(Coder eventKeyCoder, Coder eventCoder, - Duration tickerFrequency) { + PerKeyTickerGenerator( + Coder eventKeyCoder, Coder eventCoder, Duration tickerFrequency) { this.eventKeyCoder = eventKeyCoder; this.eventCoder = eventCoder; this.tickerFrequency = tickerFrequency; @@ -50,7 +68,9 @@ class PerKeyTickerGenerator @Override public @UnknownKeyFor @NonNull @Initialized PCollection>> expand( PCollection>> input) { - return input.apply("Generate Tickers", + return input + .apply( + "Generate Tickers", ParDo.of(new PerKeyTickerGeneratorDoFn<>(eventKeyCoder, tickerFrequency))) .setCoder( KvCoder.of(eventKeyCoder, KvCoder.of(VarLongCoder.of(), NullableCoder.of(eventCoder)))); @@ -59,8 +79,8 @@ class PerKeyTickerGenerator static class PerKeyTickerGeneratorDoFn extends DoFn>, KV>> { - private final static String STATE = "state"; - private final static String TIMER = "timer"; + private static final String STATE = "state"; + private static final String TIMER = "timer"; @StateId(STATE) @SuppressWarnings("unused") @@ -72,7 +92,6 @@ static class PerKeyTickerGeneratorDoFn private final Duration tickerFrequency; - PerKeyTickerGeneratorDoFn(Coder keyCoder, Duration tickerFrequency) { stateSpec = StateSpecs.value(keyCoder); this.tickerFrequency = tickerFrequency; @@ -81,8 +100,7 @@ static class PerKeyTickerGeneratorDoFn @ProcessElement public void process( @Element KV> element, - @AlwaysFetched - @StateId(STATE) ValueState state, + @AlwaysFetched @StateId(STATE) ValueState state, @TimerId(TIMER) Timer tickerTimer) { @Nullable EventKeyT keyValue = state.read(); if (keyValue != null) { diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java index 26e956e1d13e..fcd1b4b604ec 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java @@ -137,7 +137,8 @@ public KeyT getKey() { return lastCompleteGlobalSequence; } - public void setLastCompleteGlobalSequence(@Nullable ContiguousSequenceRange lastCompleteGlobalSequence) { + public void setLastCompleteGlobalSequence( + @Nullable ContiguousSequenceRange lastCompleteGlobalSequence) { this.lastCompleteGlobalSequence = lastCompleteGlobalSequence; } @@ -242,18 +243,28 @@ public int hashCode() { @Override public String toString() { - return "ProcessingState{" + - "lastOutputSequence=" + lastOutputSequence + - ", latestBufferedSequence=" + latestBufferedSequence + - ", earliestBufferedSequence=" + earliestBufferedSequence + - ", bufferedEventCount=" + bufferedEventCount + - ", lastEventReceived=" + lastEventReceived + - ", eventsReceived=" + eventsReceived + - ", duplicates=" + duplicates + - ", resultCount=" + resultCount + - ", lastCompleteGlobalSequence=" + lastCompleteGlobalSequence + - ", key=" + key + - '}'; + return "ProcessingState{" + + "lastOutputSequence=" + + lastOutputSequence + + ", latestBufferedSequence=" + + latestBufferedSequence + + ", earliestBufferedSequence=" + + earliestBufferedSequence + + ", bufferedEventCount=" + + bufferedEventCount + + ", lastEventReceived=" + + lastEventReceived + + ", eventsReceived=" + + eventsReceived + + ", duplicates=" + + duplicates + + ", resultCount=" + + resultCount + + ", lastCompleteGlobalSequence=" + + lastCompleteGlobalSequence + + ", key=" + + key + + '}'; } public boolean isProcessingCompleted() { @@ -302,7 +313,7 @@ public long resultsProducedInBundle(long numberOfResultsBeforeBundleStart) { } public void updateGlobalSequenceDetails(ContiguousSequenceRange updated) { - if(thereAreGloballySequencedEventsToBeProcessed()) { + if (thereAreGloballySequencedEventsToBeProcessed()) { // We don't update the timer if we can already process events in the onTimer batch. // Otherwise, it's possible that we will be pushing the timer to later timestamps // without a chance to run and produce output. @@ -314,8 +325,8 @@ public void updateGlobalSequenceDetails(ContiguousSequenceRange updated) { public boolean thereAreGloballySequencedEventsToBeProcessed() { return bufferedEventCount > 0 && lastCompleteGlobalSequence != null - && earliestBufferedSequence != null && - earliestBufferedSequence <= lastCompleteGlobalSequence.getEnd(); + && earliestBufferedSequence != null + && earliestBufferedSequence <= lastCompleteGlobalSequence.getEnd(); } /** @@ -369,18 +380,20 @@ public ProcessingState decode(InputStream inStream) throws IOException { long resultCount = LONG_CODER.decode(inStream); boolean isLastEventReceived = BOOLEAN_CODER.decode(inStream); KeyT key = keyCoder.decode(inStream); - ContiguousSequenceRange lastCompleteGlobalSequence = SEQUENCE_AND_TIMESTAMP_CODER.decode(inStream); - - ProcessingState result = new ProcessingState<>( - key, - lastOutputSequence, - earliestBufferedSequence, - latestBufferedSequence, - bufferedRecordCount, - recordsReceivedCount, - duplicates, - resultCount, - isLastEventReceived); + ContiguousSequenceRange lastCompleteGlobalSequence = + SEQUENCE_AND_TIMESTAMP_CODER.decode(inStream); + + ProcessingState result = + new ProcessingState<>( + key, + lastOutputSequence, + earliestBufferedSequence, + latestBufferedSequence, + bufferedRecordCount, + recordsReceivedCount, + duplicates, + resultCount, + isLastEventReceived); result.setLastCompleteGlobalSequence(lastCompleteGlobalSequence); return result; diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java index 5b24956bb8dd..1cfceab61cce 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java @@ -1,3 +1,20 @@ +/* + * 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.extensions.ordered; import java.util.Iterator; @@ -18,12 +35,12 @@ /** * Base DoFn for processing ordered events. * - * @param type of the events to process - * @param event key type + * @param type of the events to process + * @param event key type * @param state type */ -abstract class ProcessorDoFn> +abstract class ProcessorDoFn< + EventT, EventKeyT, ResultT, StateT extends MutableState> extends DoFn>, KV> { private static final Logger LOG = LoggerFactory.getLogger(ProcessorDoFn.class); @@ -45,7 +62,6 @@ abstract class ProcessorDoFn> mainOutputTupleTag, TupleTag> statusTupleTag, Duration statusUpdateFrequency, - TupleTag>>> - unprocessedEventTupleTag, + TupleTag>>> unprocessedEventTupleTag, boolean produceStatusUpdateOnEveryEvent, long maxNumberOfResultsToProduce) { this.eventExaminer = eventExaminer; @@ -78,9 +93,7 @@ public void onBundleFinish() { numberOfResultsBeforeBundleStart = null; } - /** - * @return true if each event needs to be examined. - */ + /** @return true if each event needs to be examined. */ abstract boolean checkForFirstOrLastEvent(); /** @@ -116,14 +129,13 @@ public void onBundleFinish() { .output( KV.of( processingState.getKey(), - KV.of( - currentSequence, UnprocessedEvent.create(currentEvent, Reason.duplicate)))); + KV.of(currentSequence, UnprocessedEvent.create(currentEvent, Reason.duplicate)))); return null; } StateT state; - boolean thisIsTheLastEvent = checkForFirstOrLastEvent() - && eventExaminer.isLastEvent(currentSequence, currentEvent); + boolean thisIsTheLastEvent = + checkForFirstOrLastEvent() && eventExaminer.isLastEvent(currentSequence, currentEvent); if (checkForFirstOrLastEvent() && eventExaminer.isInitialEvent(currentSequence, currentEvent)) { // First event of the key/window // What if it's a duplicate event - it will reset everything. Shall we drop/DLQ anything @@ -173,8 +185,8 @@ public void onBundleFinish() { } // Event is not ready to be processed yet - bufferEvent(currentSequence, currentEvent, processingState, bufferedEventsState, - thisIsTheLastEvent); + bufferEvent( + currentSequence, currentEvent, processingState, bufferedEventsState, thisIsTheLastEvent); // This will signal that the state hasn't been mutated. We don't need to save it. return null; @@ -204,7 +216,9 @@ protected void saveStates( } } - void processStatusTimerEvent(MultiOutputReceiver outputReceiver, Timer statusEmissionTimer, + void processStatusTimerEvent( + MultiOutputReceiver outputReceiver, + Timer statusEmissionTimer, ValueState windowClosedState, ValueState> processingStateState) { ProcessingState currentState = processingStateState.read(); @@ -250,12 +264,11 @@ protected void emitProcessingStatus( statusTimestamp); } - protected boolean reachedMaxResultCountForBundle( ProcessingState processingState, Timer largeBatchEmissionTimer) { boolean exceeded = processingState.resultsProducedInBundle( - numberOfResultsBeforeBundleStart == null ? 0 : numberOfResultsBeforeBundleStart) + numberOfResultsBeforeBundleStart == null ? 0 : numberOfResultsBeforeBundleStart) >= maxNumberOfResultsToProduce; if (exceeded) { if (LOG.isTraceEnabled()) { @@ -272,9 +285,12 @@ protected boolean reachedMaxResultCountForBundle( return exceeded; } - private void bufferEvent(long currentSequence, EventT currentEvent, + private void bufferEvent( + long currentSequence, + EventT currentEvent, ProcessingState processingState, - OrderedListState bufferedEventsState, boolean thisIsTheLastEvent) { + OrderedListState bufferedEventsState, + boolean thisIsTheLastEvent) { Instant eventTimestamp = fromLong(currentSequence); bufferedEventsState.add(TimestampedValue.of(currentEvent, eventTimestamp)); processingState.eventBuffered(currentSequence, thisIsTheLastEvent); @@ -283,10 +299,13 @@ private void bufferEvent(long currentSequence, EventT currentEvent, abstract boolean checkForSequenceGapInBufferedEvents(); @Nullable - StateT processBufferedEventRange(ProcessingState processingState, + StateT processBufferedEventRange( + ProcessingState processingState, @Nullable StateT state, - OrderedListState bufferedEventsState, MultiOutputReceiver outputReceiver, - Timer largeBatchEmissionTimer, ContiguousSequenceRange contiguousSequenceRange) { + OrderedListState bufferedEventsState, + MultiOutputReceiver outputReceiver, + Timer largeBatchEmissionTimer, + ContiguousSequenceRange contiguousSequenceRange) { Long earliestBufferedSequence = processingState.getEarliestBufferedSequence(); Long latestBufferedSequence = processingState.getLatestBufferedSequence(); if (earliestBufferedSequence == null || latestBufferedSequence == null) { @@ -296,8 +315,7 @@ StateT processBufferedEventRange(ProcessingState processingState, Instant endRange = fromLong(latestBufferedSequence + 1); // readRange is efficiently implemented and will bring records in batches - Iterable> events = - bufferedEventsState.readRange(startRange, endRange); + Iterable> events = bufferedEventsState.readRange(startRange, endRange); Instant endClearRange = startRange; // it will get re-adjusted later. @@ -328,8 +346,7 @@ StateT processBufferedEventRange(ProcessingState processingState, KV.of( processingState.getKey(), KV.of( - eventSequence, - UnprocessedEvent.create(bufferedEvent, Reason.duplicate)))); + eventSequence, UnprocessedEvent.create(bufferedEvent, Reason.duplicate)))); // TODO: When there is a large number of duplicates this can cause a situation where // we produce too much output and the runner will start throwing unrecoverable errors. // Need to add counting logic to accumulate both the normal and DLQ outputs. @@ -339,9 +356,10 @@ StateT processBufferedEventRange(ProcessingState processingState, Long lastOutputSequence = processingState.getLastOutputSequence(); boolean currentEventIsNextInSequence = lastOutputSequence != null && eventSequence == lastOutputSequence + 1; - boolean continueProcessing = checkForSequenceGapInBufferedEvents() ? - currentEventIsNextInSequence : - (eventSequence <= contiguousSequenceRange.getEnd() || currentEventIsNextInSequence); + boolean continueProcessing = + checkForSequenceGapInBufferedEvents() + ? currentEventIsNextInSequence + : (eventSequence <= contiguousSequenceRange.getEnd() || currentEventIsNextInSequence); if (!continueProcessing) { processingState.foundSequenceGap(eventSequence); // Records will be cleared up to this element diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java index b3035aebdc9d..878a0664ac87 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java @@ -1,3 +1,20 @@ +/* + * 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.extensions.ordered; import javax.annotation.Nullable; @@ -27,20 +44,26 @@ * @param result type * @param state type */ -class SequencePerKeyProcessorDoFn> +class SequencePerKeyProcessorDoFn< + EventTypeT, + EventKeyTypeT, + ResultTypeT, + StateTypeT extends MutableState> extends ProcessorDoFn { private static final Logger LOG = LoggerFactory.getLogger(SequencePerKeyProcessorDoFn.class); private static final String LARGE_BATCH_EMISSION_TIMER = "largeBatchTimer"; protected static final String BUFFERED_EVENTS = "bufferedEvents"; + @TimerId(LARGE_BATCH_EMISSION_TIMER) @SuppressWarnings("unused") private final TimerSpec largeBatchEmissionTimer = TimerSpecs.timer(TimeDomain.EVENT_TIME); + @StateId(BUFFERED_EVENTS) @SuppressWarnings("unused") private final StateSpec> bufferedEventsSpec; + @SuppressWarnings("unused") @StateId(MUTABLE_STATE) private final StateSpec> mutableStateSpec; @@ -52,6 +75,7 @@ class SequencePerKeyProcessorDoFn>> processingStateSpec; @@ -78,11 +102,16 @@ class SequencePerKeyProcessorDoFn> mainOutputTupleTag, TupleTag> statusTupleTag, Duration statusUpdateFrequency, - TupleTag>>> - unprocessedEventTupleTag, - boolean produceStatusUpdateOnEveryEvent, long maxNumberOfResultsToProduce) { - super(eventExaminer, mainOutputTupleTag, statusTupleTag, - statusUpdateFrequency, unprocessedEventTupleTag, produceStatusUpdateOnEveryEvent, + TupleTag>>> unprocessedEventTupleTag, + boolean produceStatusUpdateOnEveryEvent, + long maxNumberOfResultsToProduce) { + super( + eventExaminer, + mainOutputTupleTag, + statusTupleTag, + statusUpdateFrequency, + unprocessedEventTupleTag, + produceStatusUpdateOnEveryEvent, maxNumberOfResultsToProduce); this.bufferedEventsSpec = StateSpecs.orderedList(eventCoder); this.processingStateSpec = StateSpecs.value(ProcessingStateCoder.of(keyCoder)); @@ -104,7 +133,7 @@ boolean checkForSequenceGapInBufferedEvents() { public void processElement( @StateId(BUFFERED_EVENTS) OrderedListState bufferedEventsState, @AlwaysFetched @StateId(PROCESSING_STATE) - ValueState> processingStateState, + ValueState> processingStateState, @StateId(MUTABLE_STATE) ValueState mutableStateState, @TimerId(STATUS_EMISSION_TIMER) Timer statusEmissionTimer, @TimerId(LARGE_BATCH_EMISSION_TIMER) Timer largeBatchEmissionTimer, @@ -165,9 +194,7 @@ private boolean checkIfProcessingIsCompleted(ProcessingState proc return result; } - /** - * Process buffered events. - */ + /** Process buffered events. */ private void processBufferedEvents( ProcessingState processingState, @Nullable StateTypeT state, @@ -200,8 +227,13 @@ private void processBufferedEvents( return; } - processBufferedEventRange(processingState, state, bufferedEventsState, outputReceiver, - largeBatchEmissionTimer, ContiguousSequenceRange.EMPTY); + processBufferedEventRange( + processingState, + state, + bufferedEventsState, + outputReceiver, + largeBatchEmissionTimer, + ContiguousSequenceRange.EMPTY); } @OnTimer(LARGE_BATCH_EMISSION_TIMER) @@ -209,7 +241,7 @@ public void onBatchEmission( OnTimerContext context, @StateId(BUFFERED_EVENTS) OrderedListState bufferedEventsState, @AlwaysFetched @StateId(PROCESSING_STATE) - ValueState> processingStatusState, + ValueState> processingStatusState, @AlwaysFetched @StateId(MUTABLE_STATE) ValueState currentStateState, @TimerId(LARGE_BATCH_EMISSION_TIMER) Timer largeBatchEmissionTimer, MultiOutputReceiver outputReceiver) { @@ -249,11 +281,10 @@ public void onStatusEmission( MultiOutputReceiver outputReceiver, @TimerId(STATUS_EMISSION_TIMER) Timer statusEmissionTimer, @StateId(WINDOW_CLOSED) ValueState windowClosedState, - @StateId(PROCESSING_STATE) - ValueState> processingStateState) { + @StateId(PROCESSING_STATE) ValueState> processingStateState) { - processStatusTimerEvent(outputReceiver, statusEmissionTimer, windowClosedState, - processingStateState); + processStatusTimerEvent( + outputReceiver, statusEmissionTimer, windowClosedState, processingStateState); } @OnWindowExpiration diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java index 4d98c3b61ace..f26964670799 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java @@ -1,3 +1,20 @@ +/* + * 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.extensions.ordered.combiner; import java.util.Iterator; @@ -22,28 +39,33 @@ /** * Default global sequence combiner. - *

    - * Produces {@link ContiguousSequenceRange} of contiguous longs starting from the initial event + * + *

    Produces {@link ContiguousSequenceRange} of contiguous longs starting from the initial event * identified by {@link EventExaminer#isInitialEvent(long, EventT)}. - *

    - * This combiner currently doesn't use {@link EventExaminer#isLastEvent(long, EventT)}. + * + *

    This combiner currently doesn't use {@link EventExaminer#isLastEvent(long, EventT)}. * * @param type of key - * @param type of event - * @param type of state + * @param type of event + * @param type of state */ -public class DefaultSequenceCombiner> extends - CombineFn>>, SequenceRangeAccumulator, ContiguousSequenceRange> { +public class DefaultSequenceCombiner> + extends CombineFn< + TimestampedValue>>, + SequenceRangeAccumulator, + ContiguousSequenceRange> { private static final Logger LOG = LoggerFactory.getLogger(DefaultSequenceCombiner.class); - public static final BiFunction<@NonNull Instant, @Nullable Instant, @Nullable Instant> OLDEST_TIMESTAMP_SELECTOR = (instant1, instant2) -> { - if (instant2 == null) { - return instant1; - } - @NonNull Instant nonNullableSecondValue = instant2; - return instant1.isAfter(nonNullableSecondValue) ? instant1 : nonNullableSecondValue; - }; + public static final BiFunction<@NonNull Instant, @Nullable Instant, @Nullable Instant> + OLDEST_TIMESTAMP_SELECTOR = + (instant1, instant2) -> { + if (instant2 == null) { + return instant1; + } + @NonNull Instant nonNullableSecondValue = instant2; + return instant1.isAfter(nonNullableSecondValue) ? instant1 : nonNullableSecondValue; + }; private final EventExaminer eventExaminer; public DefaultSequenceCombiner(EventExaminer eventExaminer) { @@ -56,11 +78,13 @@ public SequenceRangeAccumulator createAccumulator() { } @Override - public SequenceRangeAccumulator addInput(SequenceRangeAccumulator accum, - TimestampedValue>> event) { + public SequenceRangeAccumulator addInput( + SequenceRangeAccumulator accum, TimestampedValue>> event) { long sequence = event.getValue().getValue().getKey(); - accum.add(sequence, event.getTimestamp(), + accum.add( + sequence, + event.getTimestamp(), eventExaminer.isInitialEvent(sequence, event.getValue().getValue().getValue())); return accum; @@ -90,7 +114,8 @@ public ContiguousSequenceRange extractOutput(SequenceRangeAccumulator accum) { @Override public @UnknownKeyFor @NonNull @Initialized Coder getAccumulatorCoder( @UnknownKeyFor @NonNull @Initialized CoderRegistry registry, - @UnknownKeyFor @NonNull @Initialized Coder>>> inputCoder) + @UnknownKeyFor @NonNull @Initialized + Coder>>> inputCoder) throws @UnknownKeyFor @NonNull @Initialized CannotProvideCoderException { return SequenceRangeAccumulatorCoder.of(); } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java index 19df464e584e..1c627ea7980f 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java @@ -1,3 +1,20 @@ +/* + * 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.extensions.ordered.combiner; import java.io.IOException; @@ -20,9 +37,7 @@ import org.checkerframework.checker.nullness.qual.UnknownKeyFor; import org.joda.time.Instant; -/** - * Default accumulator used to combine sequence ranges. - */ +/** Default accumulator used to combine sequence ranges. */ public class SequenceRangeAccumulator { private static Instant max(Instant a, Instant b) { @@ -36,7 +51,9 @@ public void add(long sequence, Instant timestamp, boolean isInitialSequence) { if (isInitialSequence && this.initialSequence != null && sequence != this.initialSequence) { throw new IllegalStateException( "There are different initial sequences detected: " - + initialSequence + " and " + sequence); + + initialSequence + + " and " + + sequence); } if (isInitialSequence) { @@ -85,9 +102,10 @@ private void clearRangesBelowInitialSequence(long sequence, Instant timestamp) { && lowerRange.getValue().getLeft() > sequence) { // The sequence is in the middle of the range. Adjust it. data.remove(lowerRange.getKey()); - data.put(sequence, - Pair.of(lowerRange.getValue().getKey(), max(timestamp, lowerRange.getValue() - .getValue()))); + data.put( + sequence, + Pair.of( + lowerRange.getValue().getKey(), max(timestamp, lowerRange.getValue().getValue()))); } data.subMap(Long.MIN_VALUE, sequence).clear(); } @@ -111,12 +129,15 @@ public int numberOfRanges() { return data.size(); } - public void merge(SequenceRangeAccumulator another) { - if (this.initialSequence != null && another.initialSequence != null + if (this.initialSequence != null + && another.initialSequence != null && !this.initialSequence.equals(another.initialSequence)) { - throw new IllegalStateException("Two accumulators contain different initial sequences: " - + this.initialSequence + " and " + another.initialSequence); + throw new IllegalStateException( + "Two accumulators contain different initial sequences: " + + this.initialSequence + + " and " + + another.initialSequence); } if (another.initialSequence != null) { @@ -129,53 +150,56 @@ public void merge(SequenceRangeAccumulator another) { } } - another.data.entrySet().forEach( - entry -> { - long lowerBound = entry.getKey(); - long upperBound = entry.getValue().getLeft(); - if (this.initialSequence != null) { - if (upperBound < initialSequence) { - // The whole range is below the initial sequence. Ignore it. - return; - } - if (lowerBound < initialSequence) { - // This will cause pruning of the range up to the initial sequence - lowerBound = this.initialSequence; - } - } - - Entry> lowerRange = this.data.floorEntry(lowerBound); - - if (lowerRange != null) { - if (lowerRange.getValue().getLeft() < lowerBound - 1) { - // Nothing to do. There is a lower non-adjacent range. - } else { - // We found an overlapping range and will replace it with a new one - upperBound = Math.max(upperBound, lowerRange.getValue().getLeft()); - lowerBound = lowerRange.getKey(); - } - } - - Entry> upperRange = this.data.floorEntry(upperBound + 1); - if (upperRange == null || - (lowerRange != null && Objects.equals(upperRange.getKey(), lowerRange.getKey()))) { - // Nothing to do - either there is no adjacent upper range or it equals the lower range - } else { - upperBound = Math.max(upperBound, upperRange.getValue().getLeft()); - } - - Instant latestTimestamp = removeAllRanges(lowerBound, upperBound, - entry.getValue().getRight()); - - this.data.put(lowerBound, Pair.of(upperBound, latestTimestamp)); - } - ); + another + .data + .entrySet() + .forEach( + entry -> { + long lowerBound = entry.getKey(); + long upperBound = entry.getValue().getLeft(); + if (this.initialSequence != null) { + if (upperBound < initialSequence) { + // The whole range is below the initial sequence. Ignore it. + return; + } + if (lowerBound < initialSequence) { + // This will cause pruning of the range up to the initial sequence + lowerBound = this.initialSequence; + } + } + + Entry> lowerRange = this.data.floorEntry(lowerBound); + + if (lowerRange != null) { + if (lowerRange.getValue().getLeft() < lowerBound - 1) { + // Nothing to do. There is a lower non-adjacent range. + } else { + // We found an overlapping range and will replace it with a new one + upperBound = Math.max(upperBound, lowerRange.getValue().getLeft()); + lowerBound = lowerRange.getKey(); + } + } + + Entry> upperRange = this.data.floorEntry(upperBound + 1); + if (upperRange == null + || (lowerRange != null + && Objects.equals(upperRange.getKey(), lowerRange.getKey()))) { + // Nothing to do - either there is no adjacent upper range or it equals the lower + // range + } else { + upperBound = Math.max(upperBound, upperRange.getValue().getLeft()); + } + + Instant latestTimestamp = + removeAllRanges(lowerBound, upperBound, entry.getValue().getRight()); + + this.data.put(lowerBound, Pair.of(upperBound, latestTimestamp)); + }); } private Instant removeAllRanges(long lowerBound, long upperBound, Instant currentTimestamp) { Instant result = currentTimestamp; - SortedMap> rangesToRemove = data.subMap(lowerBound, - upperBound); + SortedMap> rangesToRemove = data.subMap(lowerBound, upperBound); for (Pair value : rangesToRemove.values()) { result = result.isAfter(value.getRight()) ? result : value.getRight(); } @@ -207,23 +231,24 @@ public String toString() { public static class SequenceRangeAccumulatorCoder extends CustomCoder { - private static final SequenceRangeAccumulatorCoder INSTANCE = new SequenceRangeAccumulatorCoder(); + private static final SequenceRangeAccumulatorCoder INSTANCE = + new SequenceRangeAccumulatorCoder(); public static SequenceRangeAccumulatorCoder of() { return INSTANCE; } - private SequenceRangeAccumulatorCoder() { - } + private SequenceRangeAccumulatorCoder() {} private final NullableCoder initialSequenceCoder = NullableCoder.of(VarLongCoder.of()); private final VarIntCoder numberOfRangesCoder = VarIntCoder.of(); private final VarLongCoder dataCoder = VarLongCoder.of(); @Override - public void encode(SequenceRangeAccumulator value, - @UnknownKeyFor @NonNull @Initialized OutputStream outStream) - throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { + public void encode( + SequenceRangeAccumulator value, @UnknownKeyFor @NonNull @Initialized OutputStream outStream) + throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull + @Initialized IOException { numberOfRangesCoder.encode(value.numberOfRanges(), outStream); initialSequenceCoder.encode(value.initialSequence, outStream); for (Entry> entry : value.data.entrySet()) { @@ -236,7 +261,8 @@ public void encode(SequenceRangeAccumulator value, @Override public SequenceRangeAccumulator decode( @UnknownKeyFor @NonNull @Initialized InputStream inStream) - throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull @Initialized IOException { + throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull + @Initialized IOException { SequenceRangeAccumulator result = new SequenceRangeAccumulator(); int numberOfRanges = numberOfRangesCoder.decode(inStream); result.initialSequence = initialSequenceCoder.decode(inStream); diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/package-info.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/package-info.java index 8506150e0f66..4cbbca82a8cf 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/package-info.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/package-info.java @@ -17,7 +17,8 @@ */ /** * Provides a transform for ordered processing. For a detailed reference implementation which uses - * this transform visit {@link https://github.com/GoogleCloudPlatform/dataflow-ordered-processing} + * this transform visit {@link https://github.com/GoogleCloudPlatform/dataflow-ordered-processing} * * @see org.apache.beam.sdk.extensions.ordered.OrderedEventProcessor */ diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java index 66618364c345..d69d41601868 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java @@ -1,3 +1,20 @@ +/* + * 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.extensions.ordered; import java.util.ArrayList; @@ -35,12 +52,12 @@ public class OrderedEventProcessorGlobalSequenceTest extends OrderedEventProcess @org.junit.Test public void testPerfectOrderingProcessing() throws CannotProvideCoderException { Event[] events = { - Event.create(0, "id-1", "a"), - Event.create(1, "id-1", "b"), - Event.create(2, "id-1", "c"), - Event.create(3, "id-1", "d"), - Event.create(4, "id-2", "a"), - Event.create(5, "id-2", "b") + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + Event.create(2, "id-1", "c"), + Event.create(3, "id-1", "d"), + Event.create(4, "id-2", "a"), + Event.create(5, "id-2", "b") }; Collection> expectedOutput = new ArrayList<>(); @@ -63,15 +80,15 @@ public void testPerfectOrderingProcessing() throws CannotProvideCoderException { @Test public void testOutOfSequenceProcessing() throws CannotProvideCoderException { Event[] events = { - Event.create(2, "id-1", "c"), - Event.create(1, "id-1", "b"), - Event.create(0, "id-1", "a"), - Event.create(3, "id-1", "d"), - Event.create(5, "id-2", "b"), - Event.create(6, "id-2", "c"), - Event.create(8, "id-2", "e"), - Event.create(4, "id-2", "a"), - Event.create(7, "id-2", "d") + Event.create(2, "id-1", "c"), + Event.create(1, "id-1", "b"), + Event.create(0, "id-1", "a"), + Event.create(3, "id-1", "d"), + Event.create(5, "id-2", "b"), + Event.create(6, "id-2", "c"), + Event.create(8, "id-2", "e"), + Event.create(4, "id-2", "a"), + Event.create(7, "id-2", "d") }; Collection> expectedOutput = new ArrayList<>(); @@ -97,19 +114,18 @@ public void testOutOfSequenceProcessing() throws CannotProvideCoderException { @Test public void testHandlingOfDuplicateSequences() throws CannotProvideCoderException { Event[] events = { - Event.create(3, "id-1", "d"), - Event.create(2, "id-1", "c"), - - // Duplicates - Event.create(3, "id-1", "d"), - Event.create(3, "id-1", "d"), - - Event.create(0, "id-1", "a"), - Event.create(1, "id-1", "b"), - - // Additional duplicates - Event.create(1, "id-1", "b"), - Event.create(3, "id-1", "d"), + Event.create(3, "id-1", "d"), + Event.create(2, "id-1", "c"), + + // Duplicates + Event.create(3, "id-1", "d"), + Event.create(3, "id-1", "d"), + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + + // Additional duplicates + Event.create(1, "id-1", "b"), + Event.create(3, "id-1", "d"), }; Collection> expectedOutput = new ArrayList<>(); @@ -137,15 +153,14 @@ public void testHandlingOfDuplicateSequences() throws CannotProvideCoderExceptio @Test public void testTreatingSequencesBelowInitialAsDuplicates() throws CannotProvideCoderException { Event[] events = { - Event.create(3, "id-1", "d"), - Event.create(2, "id-1", "c"), - - // Earlier events - Event.create(-1, "id-1", "early-1"), - Event.create(-2, "id-1", "early-2"), - - Event.create(0, "id-1", "a"), - Event.create(1, "id-1", "b") + Event.create(3, "id-1", "d"), + Event.create(2, "id-1", "c"), + + // Earlier events + Event.create(-1, "id-1", "early-1"), + Event.create(-2, "id-1", "early-2"), + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b") }; Collection> expectedOutput = new ArrayList<>(); @@ -171,10 +186,10 @@ public void testTreatingSequencesBelowInitialAsDuplicates() throws CannotProvide @Test public void testHandlingOfCheckedExceptions() throws CannotProvideCoderException { Event[] events = { - Event.create(0, "id-1", "a"), - Event.create(1, "id-1", "b"), - Event.create(2, "id-1", StringBuilderState.BAD_VALUE), - Event.create(3, "id-1", "c"), + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + Event.create(2, "id-1", StringBuilderState.BAD_VALUE), + Event.create(3, "id-1", "c"), }; // This is an interesting case - even though event #2 is not processed it doesn't affect @@ -209,12 +224,12 @@ public void testHandlingOfCheckedExceptions() throws CannotProvideCoderException @Test public void testProcessingWithEveryOtherResultEmission() throws CannotProvideCoderException { Event[] events = { - Event.create(2, "id-1", "c"), - Event.create(1, "id-1", "b"), - Event.create(0, "id-1", "a"), - Event.create(3, "id-1", "d"), - Event.create(4, "id-2", "a"), - Event.create(5, "id-2", "b"), + Event.create(2, "id-1", "c"), + Event.create(1, "id-1", "b"), + Event.create(0, "id-1", "a"), + Event.create(3, "id-1", "d"), + Event.create(4, "id-2", "a"), + Event.create(5, "id-2", "b"), }; Collection> expectedOutput = new ArrayList<>(); @@ -256,7 +271,6 @@ public void testLargeBufferedOutputInTimer() throws CannotProvideCoderException events.add(Event.create(sequence, key, outputPerElement)); output.append(outputPerElement); expectedOutput.add(KV.of(key, output.toString())); - } testGlobalSequenceProcessing( @@ -272,7 +286,7 @@ public void testLargeBufferedOutputInTimer() throws CannotProvideCoderException public void testSequenceGapProcessingInBufferedOutput() throws CannotProvideCoderException { int maxResultsPerOutput = 3; - long[] sequences = new long[]{2, 3, 7, 8, 9, 10, 1, 4, 5, 6}; + long[] sequences = new long[] {2, 3, 7, 8, 9, 10, 1, 4, 5, 6}; List events = new ArrayList<>(sequences.length); List> expectedOutput = new ArrayList<>(sequences.length); @@ -284,11 +298,13 @@ public void testSequenceGapProcessingInBufferedOutput() throws CannotProvideCode } StringBuilder output = new StringBuilder(); - Arrays.stream(sequences).sorted().forEach(sequence -> { - output.append(sequence + "-"); - expectedOutput.add(KV.of(key, output.toString())); - } - ); + Arrays.stream(sequences) + .sorted() + .forEach( + sequence -> { + output.append(sequence + "-"); + expectedOutput.add(KV.of(key, output.toString())); + }); testGlobalSequenceProcessing( events.toArray(new Event[events.size()]), @@ -302,10 +318,10 @@ public void testSequenceGapProcessingInBufferedOutput() throws CannotProvideCode @Test public void testHandlingOfMaxSequenceNumber() throws CannotProvideCoderException { Event[] events = { - Event.create(1, "id-1", "b"), - Event.create(0, "id-1", "a"), - Event.create(Long.MAX_VALUE, "id-1", "d"), - Event.create(2, "id-1", "c") + Event.create(1, "id-1", "b"), + Event.create(0, "id-1", "a"), + Event.create(Long.MAX_VALUE, "id-1", "d"), + Event.create(2, "id-1", "c") }; Collection> expectedOutput = new ArrayList<>(); @@ -336,9 +352,9 @@ public void testHandlingOfMaxSequenceNumber() throws CannotProvideCoderException public void testProcessingOfTheLastInput() throws CannotProvideCoderException { // TODO: fix the test. Need to see that the resulting status reflects the last input Event[] events = { - Event.create(0, "id-1", "a"), - Event.create(1, "id-1", "b"), - Event.create(2, "id-1", StringEventExaminer.LAST_INPUT) + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + Event.create(2, "id-1", StringEventExaminer.LAST_INPUT) }; Collection> expectedOutput = new ArrayList<>(); @@ -355,7 +371,6 @@ public void testProcessingOfTheLastInput() throws CannotProvideCoderException { ContiguousSequenceRange.of(0, 2, new Instant())); } - private void testGlobalSequenceProcessing( Event[] events, Collection> expectedOutput, @@ -370,7 +385,8 @@ private void testGlobalSequenceProcessing( NO_EXPECTED_DLQ_EVENTS, emissionFrequency, initialSequence, - maxResultsPerOutput, expectedLastCompleteRange); + maxResultsPerOutput, + expectedLastCompleteRange); } private void testGlobalSequenceProcessing( @@ -394,7 +410,8 @@ private void testGlobalSequenceProcessing( maxResultsPerOutput, false /* produceStatusOnEveryEvent */, STREAMING, - GLOBAL_SEQUENCE, expectedLastCompleteRange); + GLOBAL_SEQUENCE, + expectedLastCompleteRange); } // Test a batch pipeline @@ -419,7 +436,6 @@ private void testGlobalSequenceProcessing( } } - @Test public void testWindowedProcessing() throws CannotProvideCoderException { @@ -487,21 +503,24 @@ public void testWindowedProcessing() throws CannotProvideCoderException { .containsInAnyOrder(KV.of("id-1", "c"), KV.of("id-1", "cd")); // TODO: can we make the status assertions work? -// PAssert.that("Statuses match in window 1", processingResult.processingStatuses()) -// .inWindow(window1) -// .containsInAnyOrder( -//// KV.of("id-1", OrderedProcessingStatus.create(0L, 0, null, null, 1, 1, 0, false)), -// KV.of("id-1", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2, 0, false)), -//// KV.of("id-2", OrderedProcessingStatus.create(0L, 0, null, null, 1, 1, 0, false)), -//// KV.of("id-2", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2, 0, false)), -// KV.of("id-2", OrderedProcessingStatus.create(2L, 0, null, null, 3, 3, 0, false)) -// ); - -// PAssert.that("Statuses match in window 2", processingResult.processingStatuses()) -// .inWindow(window2) -// .containsInAnyOrder( -// KV.of("id-1", OrderedProcessingStatus.create(0L, 0, null, null, 1, 1, 0, false)), -// KV.of("id-1", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2, 0, false))); + // PAssert.that("Statuses match in window 1", processingResult.processingStatuses()) + // .inWindow(window1) + // .containsInAnyOrder( + //// KV.of("id-1", OrderedProcessingStatus.create(0L, 0, null, null, 1, 1, 0, + // false)), + // KV.of("id-1", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2, 0, false)), + //// KV.of("id-2", OrderedProcessingStatus.create(0L, 0, null, null, 1, 1, 0, + // false)), + //// KV.of("id-2", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2, 0, + // false)), + // KV.of("id-2", OrderedProcessingStatus.create(2L, 0, null, null, 3, 3, 0, false)) + // ); + + // PAssert.that("Statuses match in window 2", processingResult.processingStatuses()) + // .inWindow(window2) + // .containsInAnyOrder( + // KV.of("id-1", OrderedProcessingStatus.create(0L, 0, null, null, 1, 1, 0, false)), + // KV.of("id-1", OrderedProcessingStatus.create(1L, 0, null, null, 2, 2, 0, false))); PAssert.that("Unprocessed events match", processingResult.unprocessedEvents()) .containsInAnyOrder(NO_EXPECTED_DLQ_EVENTS); diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java index 818063015825..6909a3bb992c 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java @@ -1,3 +1,20 @@ +/* + * 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.extensions.ordered; import java.util.ArrayList; @@ -25,12 +42,12 @@ public class OrderedEventProcessorPerKeySequenceTest extends OrderedEventProcess @Test public void testPerfectOrderingProcessing() throws CannotProvideCoderException { Event[] events = { - Event.create(0, "id-1", "a"), - Event.create(1, "id-1", "b"), - Event.create(2, "id-1", "c"), - Event.create(3, "id-1", "d"), - Event.create(0, "id-2", "a"), - Event.create(1, "id-2", "b") + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + Event.create(2, "id-1", "c"), + Event.create(3, "id-1", "d"), + Event.create(0, "id-2", "a"), + Event.create(1, "id-2", "b") }; Collection> expectedStatuses = new ArrayList<>(); @@ -80,15 +97,15 @@ public void testPerfectOrderingProcessing() throws CannotProvideCoderException { @Test public void testOutOfSequenceProcessing() throws CannotProvideCoderException { Event[] events = { - Event.create(2, "id-1", "c"), - Event.create(1, "id-1", "b"), - Event.create(0, "id-1", "a"), - Event.create(3, "id-1", "d"), - Event.create(1, "id-2", "b"), - Event.create(2, "id-2", "c"), - Event.create(4, "id-2", "e"), - Event.create(0, "id-2", "a"), - Event.create(3, "id-2", "d") + Event.create(2, "id-1", "c"), + Event.create(1, "id-1", "b"), + Event.create(0, "id-1", "a"), + Event.create(3, "id-1", "d"), + Event.create(1, "id-2", "b"), + Event.create(2, "id-2", "c"), + Event.create(4, "id-2", "e"), + Event.create(0, "id-2", "a"), + Event.create(3, "id-2", "d") }; Collection> expectedStatuses = new ArrayList<>(); @@ -141,12 +158,12 @@ public void testOutOfSequenceProcessing() throws CannotProvideCoderException { @Test public void testUnfinishedProcessing() throws CannotProvideCoderException { Event[] events = { - Event.create(2, "id-1", "c"), - // Excluded Event.create(1, "id-1", "b"), - Event.create(0, "id-1", "a"), - Event.create(3, "id-1", "d"), - Event.create(0, "id-2", "a"), - Event.create(1, "id-2", "b"), + Event.create(2, "id-1", "c"), + // Excluded Event.create(1, "id-1", "b"), + Event.create(0, "id-1", "a"), + Event.create(3, "id-1", "d"), + Event.create(0, "id-2", "a"), + Event.create(1, "id-2", "b"), }; Collection> expectedStatuses = new ArrayList<>(); @@ -166,17 +183,17 @@ public void testUnfinishedProcessing() throws CannotProvideCoderException { @Test public void testHandlingOfDuplicateSequences() throws CannotProvideCoderException { Event[] events = { - Event.create(3, "id-1", "d"), - Event.create(2, "id-1", "c"), - // Duplicates to be buffered - Event.create(3, "id-1", "d"), - Event.create(3, "id-1", "d"), - Event.create(0, "id-1", "a"), - Event.create(1, "id-1", "b"), - - // Duplicates after the events are processed - Event.create(1, "id-1", "b"), - Event.create(3, "id-1", "d"), + Event.create(3, "id-1", "d"), + Event.create(2, "id-1", "c"), + // Duplicates to be buffered + Event.create(3, "id-1", "d"), + Event.create(3, "id-1", "d"), + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + + // Duplicates after the events are processed + Event.create(1, "id-1", "b"), + Event.create(3, "id-1", "d"), }; int resultCount = 4; int duplicateCount = 4; @@ -214,10 +231,10 @@ public void testHandlingOfDuplicateSequences() throws CannotProvideCoderExceptio @Test public void testHandlingOfCheckedExceptions() throws CannotProvideCoderException { Event[] events = { - Event.create(0, "id-1", "a"), - Event.create(1, "id-1", "b"), - Event.create(2, "id-1", StringBuilderState.BAD_VALUE), - Event.create(3, "id-1", "c"), + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + Event.create(2, "id-1", StringBuilderState.BAD_VALUE), + Event.create(3, "id-1", "c"), }; Collection> expectedStatuses = new ArrayList<>(); @@ -250,12 +267,12 @@ public void testHandlingOfCheckedExceptions() throws CannotProvideCoderException @Test public void testProcessingWithEveryOtherResultEmission() throws CannotProvideCoderException { Event[] events = { - Event.create(2, "id-1", "c"), - Event.create(1, "id-1", "b"), - Event.create(0, "id-1", "a"), - Event.create(3, "id-1", "d"), - Event.create(0, "id-2", "a"), - Event.create(1, "id-2", "b"), + Event.create(2, "id-1", "c"), + Event.create(1, "id-1", "b"), + Event.create(0, "id-1", "a"), + Event.create(3, "id-1", "d"), + Event.create(0, "id-2", "a"), + Event.create(1, "id-2", "b"), }; Collection> expectedStatuses = new ArrayList<>(); @@ -367,7 +384,7 @@ public void testLargeBufferedOutputInTimer() throws CannotProvideCoderException public void testSequenceGapProcessingInBufferedOutput() throws CannotProvideCoderException { int maxResultsPerOutput = 3; - long[] sequences = new long[]{2, 3, 7, 8, 9, 10, 1, 4, 5, 6}; + long[] sequences = new long[] {2, 3, 7, 8, 9, 10, 1, 4, 5, 6}; List events = new ArrayList<>(sequences.length); List> expectedOutput = new ArrayList<>(sequences.length); @@ -461,9 +478,9 @@ public void testSequenceGapProcessingInBufferedOutput() throws CannotProvideCode @Test public void testHandlingOfMaxSequenceNumber() throws CannotProvideCoderException { Event[] events = { - Event.create(0, "id-1", "a"), - Event.create(1, "id-1", "b"), - Event.create(Long.MAX_VALUE, "id-1", "c") + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + Event.create(Long.MAX_VALUE, "id-1", "c") }; Collection> expectedStatuses = new ArrayList<>(); @@ -497,9 +514,9 @@ public void testHandlingOfMaxSequenceNumber() throws CannotProvideCoderException @Test public void testProcessingOfTheLastInput() throws CannotProvideCoderException { Event[] events = { - Event.create(0, "id-1", "a"), - Event.create(1, "id-1", "b"), - Event.create(2, "id-1", StringEventExaminer.LAST_INPUT) + Event.create(0, "id-1", "a"), + Event.create(1, "id-1", "b"), + Event.create(2, "id-1", StringEventExaminer.LAST_INPUT) }; Collection> expectedStatuses = new ArrayList<>(); @@ -565,7 +582,8 @@ protected void testPerKeySequenceProcessing( maxResultsPerOutput, produceStatusOnEveryEvent, STREAMING, - false, ContiguousSequenceRange.EMPTY); + false, + ContiguousSequenceRange.EMPTY); // Test a batch pipeline doTest( @@ -578,7 +596,8 @@ protected void testPerKeySequenceProcessing( maxResultsPerOutput, produceStatusOnEveryEvent, BATCH, - false, ContiguousSequenceRange.EMPTY); + false, + ContiguousSequenceRange.EMPTY); } @Test diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTestBase.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTestBase.java index 79f874ce5961..fd651b919df1 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTestBase.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTestBase.java @@ -77,10 +77,8 @@ public class OrderedEventProcessorTestBase { public static final boolean BATCH = false; public static final Set>>> NO_EXPECTED_DLQ_EVENTS = Collections.emptySet(); - @Rule - public final transient TestPipeline streamingPipeline = TestPipeline.create(); - @Rule - public final transient TestPipeline batchPipeline = TestPipeline.create(); + @Rule public final transient TestPipeline streamingPipeline = TestPipeline.create(); + @Rule public final transient TestPipeline batchPipeline = TestPipeline.create(); protected boolean runTestsOnDataflowRunner() { return Boolean.getBoolean("run-tests-on-dataflow"); @@ -88,7 +86,7 @@ protected boolean runTestsOnDataflowRunner() { protected String getSystemProperty(String name) { String property = System.getProperty(name); - if(property == null) { + if (property == null) { throw new IllegalStateException("Unable to find system property '" + name + "'"); } return property; @@ -136,7 +134,8 @@ protected void doTest( Pipeline pipeline = streaming ? streamingPipeline : batchPipeline; if (runTestsOnDataflowRunner()) { pipeline.getOptions().setRunner(TestDataflowRunner.class); - TestDataflowPipelineOptions options = pipeline.getOptions().as(TestDataflowPipelineOptions.class); + TestDataflowPipelineOptions options = + pipeline.getOptions().as(TestDataflowPipelineOptions.class); options.setExperiments(Arrays.asList("disable_runner_v2")); options.setTempRoot("gs://" + getSystemProperty("temp_dataflow_bucket")); } @@ -148,10 +147,10 @@ protected void doTest( rawInput.apply("To KV", ParDo.of(new MapEventsToKV())); OrderedProcessingHandler handler = - isGlobalSequence ? - new StringBufferOrderedProcessingWithGlobalSequenceHandler(emissionFrequency, - initialSequence) : - new StringBufferOrderedProcessingHandler(emissionFrequency, initialSequence); + isGlobalSequence + ? new StringBufferOrderedProcessingWithGlobalSequenceHandler( + emissionFrequency, initialSequence) + : new StringBufferOrderedProcessingHandler(emissionFrequency, initialSequence); handler.setMaxOutputElementsPerBundle(maxResultsPerOutput); if (produceStatusOnEveryEvent) { handler.setProduceStatusUpdateOnEveryEvent(true); @@ -171,7 +170,8 @@ protected void doTest( PAssert.that("Output matches", processingResult.output()).containsInAnyOrder(expectedOutput); if (streaming && expectedStatuses != null) { - // Only in a streaming pipeline the events will arrive in a pre-determined order and the statuses + // Only in a streaming pipeline the events will arrive in a pre-determined order and the + // statuses // will be deterministic. In batch pipelines events can be processed in any order, // so we skip status verification and rely on the output and unprocessed event matches. PAssert.that("Statuses match", processingResult.processingStatuses()) @@ -205,22 +205,22 @@ protected void doTest( .containsInAnyOrder(expectedUnprocessedEvents); } - if (expectedLastCompletedSequence != null - && processingResult.latestContiguousRange() != null) { - PCollection globalSequences = rawInput - .apply("Publish Global Sequences", - new GlobalSequenceRangePublisher(processingResult.latestContiguousRange(), + if (expectedLastCompletedSequence != null && processingResult.latestContiguousRange() != null) { + PCollection globalSequences = + rawInput.apply( + "Publish Global Sequences", + new GlobalSequenceRangePublisher( + processingResult.latestContiguousRange(), handler.getKeyCoder(pipeline, input.getCoder()), handler.getEventCoder(pipeline, input.getCoder()))); - PAssert.that("CompletedSequenceRange verification", globalSequences).satisfies( - new LastExpectedGlobalSequenceRangeMatcher(expectedLastCompletedSequence) - ); + PAssert.that("CompletedSequenceRange verification", globalSequences) + .satisfies(new LastExpectedGlobalSequenceRangeMatcher(expectedLastCompletedSequence)); } pipeline.run(); } - static class LastExpectedGlobalSequenceRangeMatcher implements - SerializableFunction, Void> { + static class LastExpectedGlobalSequenceRangeMatcher + implements SerializableFunction, Void> { private final long expectedStart; private final long expectedEnd; @@ -244,13 +244,19 @@ public Void apply(Iterable input) { listOfRanges.append(lastRange); } listOfRanges.append(']'); - boolean foundExpectedRange = lastRange != null && - lastRange.getStart() == expectedStart && lastRange.getEnd() == expectedEnd; + boolean foundExpectedRange = + lastRange != null + && lastRange.getStart() == expectedStart + && lastRange.getEnd() == expectedEnd; assertThat( - "Expected range not found: [" + expectedStart + '-' + expectedEnd + "Expected range not found: [" + + expectedStart + + '-' + + expectedEnd + "], received ranges: " - + listOfRanges, foundExpectedRange); + + listOfRanges, + foundExpectedRange); return null; } } @@ -311,7 +317,7 @@ public boolean matches(Object actual) { && originalEvent.getEvent().equals(eventToMatch.getEvent()) && originalEvent.getReason() == eventToMatch.getReason() && normalizeExplanation(originalEvent.getExplanation()) - .equals(normalizeExplanation(eventToMatch.getExplanation())); + .equals(normalizeExplanation(eventToMatch.getExplanation())); } @Override @@ -331,8 +337,8 @@ static String normalizeExplanation(String value) { } } - static class GlobalSequenceRangePublisher extends - PTransform, PCollection> { + static class GlobalSequenceRangePublisher + extends PTransform, PCollection> { private final PCollectionView lastCompletedSequenceRangeView; private final Coder keyCoder; @@ -340,7 +346,8 @@ static class GlobalSequenceRangePublisher extends public GlobalSequenceRangePublisher( PCollectionView latestCompletedSequenceRange, - Coder keyCoder, Coder eventCoder) { + Coder keyCoder, + Coder eventCoder) { this.lastCompletedSequenceRangeView = latestCompletedSequenceRange; this.keyCoder = keyCoder; this.eventCoder = eventCoder; @@ -348,30 +355,34 @@ public GlobalSequenceRangePublisher( @Override public PCollection expand(PCollection input) { - PCollection>> events = input - // In production pipelines the global sequence will typically be obtained - // by using GenerateSequence. But GenerateSequence doesn't work well with TestStream, - // That's why we use the input events here. -// .apply("Create Ticker", -// GenerateSequence.from(0).to(2).withRate(1, Duration.standardSeconds(5))) - .apply("To KV", ParDo.of(new MapEventsToKV())); + PCollection>> events = + input + // In production pipelines the global sequence will typically be obtained + // by using GenerateSequence. But GenerateSequence doesn't work well with TestStream, + // That's why we use the input events here. + // .apply("Create Ticker", + // GenerateSequence.from(0).to(2).withRate(1, + // Duration.standardSeconds(5))) + .apply("To KV", ParDo.of(new MapEventsToKV())); if (input.isBounded() == IsBounded.BOUNDED) { - return events.apply("Emit SideInput", ParDo.of(new SideInputEmitter()) - .withSideInput("lastCompletedSequence", lastCompletedSequenceRangeView)); + return events.apply( + "Emit SideInput", + ParDo.of(new SideInputEmitter()) + .withSideInput("lastCompletedSequence", lastCompletedSequenceRangeView)); } else { - PCollection>> tickers = events - .apply("Create Tickers", - new PerKeyTickerGenerator<>(keyCoder, eventCoder, - Duration.standardSeconds(1))); - return - tickers - .apply("Emit SideInput", ParDo.of(new SideInputEmitter()) - .withSideInput("lastCompletedSequence", lastCompletedSequenceRangeView)); + PCollection>> tickers = + events.apply( + "Create Tickers", + new PerKeyTickerGenerator<>(keyCoder, eventCoder, Duration.standardSeconds(1))); + return tickers.apply( + "Emit SideInput", + ParDo.of(new SideInputEmitter()) + .withSideInput("lastCompletedSequence", lastCompletedSequenceRangeView)); } } - static class SideInputEmitter extends - DoFn>, ContiguousSequenceRange> { + static class SideInputEmitter + extends DoFn>, ContiguousSequenceRange> { @ProcessElement public void produceCompletedRange( @@ -381,4 +392,4 @@ public void produceCompletedRange( } } } -} \ No newline at end of file +} diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/StringBufferOrderedProcessingHandler.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/StringBufferOrderedProcessingHandler.java index aed3e78b9de1..1da46c3262e4 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/StringBufferOrderedProcessingHandler.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/StringBufferOrderedProcessingHandler.java @@ -32,8 +32,8 @@ public static class StringBufferOrderedProcessingWithGlobalSequenceHandler private final EventExaminer eventExaminer; - public StringBufferOrderedProcessingWithGlobalSequenceHandler(int emissionFrequency, - long initialSequence) { + public StringBufferOrderedProcessingWithGlobalSequenceHandler( + int emissionFrequency, long initialSequence) { super(String.class, String.class, StringBuilderState.class, String.class); this.eventExaminer = new StringEventExaminer(initialSequence, emissionFrequency); } diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorCoderTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorCoderTest.java index d2f0a7127818..0e5b0b7c819a 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorCoderTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorCoderTest.java @@ -1,3 +1,20 @@ +/* + * 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.extensions.ordered.combiner; import static org.junit.Assert.assertEquals; @@ -44,13 +61,11 @@ public void testEncodingAccumulatorWithInitialSequence() throws IOException { doTestEncodingAndDecoding(accumulator); } - private void doTestEncodingAndDecoding(SequenceRangeAccumulator value) - throws IOException { + private void doTestEncodingAndDecoding(SequenceRangeAccumulator value) throws IOException { ByteArrayOutputStream output = new ByteArrayOutputStream(); coder.encode(value, output); SequenceRangeAccumulator decoded = coder.decode(new ByteArrayInputStream(output.toByteArray())); assertEquals("Accumulator", value, decoded); } - } diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java index ff3b15142a58..e92a2ef517b6 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java @@ -1,3 +1,20 @@ +/* + * 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.extensions.ordered.combiner; import java.util.Arrays; @@ -10,7 +27,7 @@ public class SequenceRangeAccumulatorTest { // Atomic just in case tests are run in parallel - private final static AtomicLong currentTicker = new AtomicLong(); + private static final AtomicLong currentTicker = new AtomicLong(); static Instant nextTimestamp() { return Instant.ofEpochMilli(currentTicker.getAndIncrement()); @@ -44,22 +61,24 @@ static class Event { @Test public void testSimpleAccumulation() { - Event[] events = new Event[]{ - new Event(1, nextTimestamp(), true), - new Event(2, nextTimestamp()), - new Event(3, nextTimestamp()) - }; + Event[] events = + new Event[] { + new Event(1, nextTimestamp(), true), + new Event(2, nextTimestamp()), + new Event(3, nextTimestamp()) + }; doTestAccumulation(events, ContiguousSequenceRange.of(1, 3, eventTimestamp(events, 3)), 1); } @Test public void testReverseArrivalHandling() { - Event[] events = new Event[]{ - new Event(3, nextTimestamp()), - new Event(2, nextTimestamp()), - new Event(1, nextTimestamp(), true) - }; + Event[] events = + new Event[] { + new Event(3, nextTimestamp()), + new Event(2, nextTimestamp()), + new Event(1, nextTimestamp(), true) + }; Instant timestampOfEventNumber1 = eventTimestamp(events, 1); doTestAccumulation(events, ContiguousSequenceRange.of(1, 3, timestampOfEventNumber1), 1); @@ -67,66 +86,69 @@ public void testReverseArrivalHandling() { @Test public void testPartialRangeAccumulation() { - Event[] events = new Event[]{ - new Event(1, nextTimestamp(), true), - new Event(2, nextTimestamp()), - new Event(3, nextTimestamp()), - new Event(5, nextTimestamp()), - new Event(7, nextTimestamp()), - }; + Event[] events = + new Event[] { + new Event(1, nextTimestamp(), true), + new Event(2, nextTimestamp()), + new Event(3, nextTimestamp()), + new Event(5, nextTimestamp()), + new Event(7, nextTimestamp()), + }; doTestAccumulation(events, ContiguousSequenceRange.of(1, 3, eventTimestamp(events, 3)), 3); } @Test public void testMergingRangeAccumulation() { - Event[] events = new Event[]{ - new Event(1, nextTimestamp(), true), - new Event(2, nextTimestamp()), - new Event(3, nextTimestamp()), - new Event(5, nextTimestamp()), - new Event(7, nextTimestamp()), - new Event(6, nextTimestamp()), - }; + Event[] events = + new Event[] { + new Event(1, nextTimestamp(), true), + new Event(2, nextTimestamp()), + new Event(3, nextTimestamp()), + new Event(5, nextTimestamp()), + new Event(7, nextTimestamp()), + new Event(6, nextTimestamp()), + }; doTestAccumulation(events, ContiguousSequenceRange.of(1, 3, eventTimestamp(events, 3)), 2); } @Test public void testNoStartEvent() { - Event[] events = new Event[]{ - new Event(2, nextTimestamp()), - new Event(3, nextTimestamp()), - new Event(1, nextTimestamp()), - - new Event(5, nextTimestamp()), - }; + Event[] events = + new Event[] { + new Event(2, nextTimestamp()), + new Event(3, nextTimestamp()), + new Event(1, nextTimestamp()), + new Event(5, nextTimestamp()), + }; doTestAccumulation(events, ContiguousSequenceRange.EMPTY, 2); } @Test public void testNoEventsAccumulation() { - Event[] events = new Event[]{}; + Event[] events = new Event[] {}; doTestAccumulation(events, ContiguousSequenceRange.EMPTY, 0); } @Test public void testRemovingRangesBelowInitialSequenceDuringAccumulation() { - Event[] events = new Event[]{ - // First range - new Event(2, nextTimestamp()), - new Event(3, nextTimestamp()), - new Event(1, nextTimestamp()), + Event[] events = + new Event[] { + // First range + new Event(2, nextTimestamp()), + new Event(3, nextTimestamp()), + new Event(1, nextTimestamp()), - // Second range - new Event(5, nextTimestamp()), - new Event(6, nextTimestamp()), + // Second range + new Event(5, nextTimestamp()), + new Event(6, nextTimestamp()), - // This event should prune everything below - new Event(7, nextTimestamp(), true), - }; + // This event should prune everything below + new Event(7, nextTimestamp(), true), + }; doTestAccumulation(events, ContiguousSequenceRange.of(7, 7, eventTimestamp(events, 7)), 1); } @@ -134,43 +156,42 @@ public void testRemovingRangesBelowInitialSequenceDuringAccumulation() { @Test public void testRemovingElementsBelowInitialSequenceDuringAccumulation() { - Event[] events = new Event[]{ - // First range - new Event(2, nextTimestamp()), - new Event(3, nextTimestamp()), - new Event(1, nextTimestamp()), + Event[] events = + new Event[] { + // First range + new Event(2, nextTimestamp()), + new Event(3, nextTimestamp()), + new Event(1, nextTimestamp()), - // Second range - new Event(5, nextTimestamp()), - new Event(6, nextTimestamp()), - new Event(7, nextTimestamp()), - new Event(8, nextTimestamp()), + // Second range + new Event(5, nextTimestamp()), + new Event(6, nextTimestamp()), + new Event(7, nextTimestamp()), + new Event(8, nextTimestamp()), - // This event should reduce the range. - new Event(7, nextTimestamp(), true), - }; + // This event should reduce the range. + new Event(7, nextTimestamp(), true), + }; Instant timestampOfTheLastEvent = events[events.length - 1].timestamp; doTestAccumulation(events, ContiguousSequenceRange.of(7, 8, timestampOfTheLastEvent), 1); } - private static void doTestAccumulation(Event[] events, ContiguousSequenceRange expectedResult, - int expectedNumberOfRanges) { + private static void doTestAccumulation( + Event[] events, ContiguousSequenceRange expectedResult, int expectedNumberOfRanges) { SequenceRangeAccumulator accumulator = new SequenceRangeAccumulator(); Arrays.stream(events).forEach(e -> accumulator.add(e.sequence, e.timestamp, e.initialEvent)); - Assert.assertEquals("Accumulated results", - expectedResult, - accumulator.largestContinuousRange()); + Assert.assertEquals( + "Accumulated results", expectedResult, accumulator.largestContinuousRange()); Assert.assertEquals("Number of ranges", expectedNumberOfRanges, accumulator.numberOfRanges()); } - @Test public void testEmptyMerge() { - Event[] set1 = new Event[]{}; - Event[] set2 = new Event[]{}; + Event[] set1 = new Event[] {}; + Event[] set2 = new Event[] {}; ContiguousSequenceRange expectedResult = ContiguousSequenceRange.EMPTY; int expectedNumberOfRanges = 0; @@ -180,15 +201,16 @@ public void testEmptyMerge() { @Test public void testMergingNonEmptyWithEmpty() { - Event[] set1 = new Event[]{ - new Event(3, nextTimestamp()), - new Event(2, nextTimestamp()), - new Event(1, nextTimestamp(), true) - }; - Event[] set2 = new Event[]{}; - - ContiguousSequenceRange expectedResult = ContiguousSequenceRange.of(1, 3, - eventTimestamp(set1, 1L)); + Event[] set1 = + new Event[] { + new Event(3, nextTimestamp()), + new Event(2, nextTimestamp()), + new Event(1, nextTimestamp(), true) + }; + Event[] set2 = new Event[] {}; + + ContiguousSequenceRange expectedResult = + ContiguousSequenceRange.of(1, 3, eventTimestamp(set1, 1L)); int expectedNumberOfRanges = 1; doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); @@ -196,18 +218,19 @@ public void testMergingNonEmptyWithEmpty() { @Test public void testMergingWithLowerNonAdjacentRange() { - Event[] set1 = new Event[]{ - new Event(1, nextTimestamp(), true), - new Event(2, nextTimestamp()), - }; - Event[] set2 = new Event[]{ - new Event(4, nextTimestamp()), - new Event(5, nextTimestamp()), - new Event(6, nextTimestamp()) - }; - - ContiguousSequenceRange expectedResult = ContiguousSequenceRange.of(1, 2, - eventTimestamp(set1, 2L)); + Event[] set1 = + new Event[] { + new Event(1, nextTimestamp(), true), new Event(2, nextTimestamp()), + }; + Event[] set2 = + new Event[] { + new Event(4, nextTimestamp()), + new Event(5, nextTimestamp()), + new Event(6, nextTimestamp()) + }; + + ContiguousSequenceRange expectedResult = + ContiguousSequenceRange.of(1, 2, eventTimestamp(set1, 2L)); int expectedNumberOfRanges = 2; doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); @@ -215,15 +238,16 @@ public void testMergingWithLowerNonAdjacentRange() { @Test public void testMergingWithoutAnyInitialEvents() { - Event[] set1 = new Event[]{ - new Event(1, nextTimestamp()), - new Event(2, nextTimestamp()), - }; - Event[] set2 = new Event[]{ - new Event(4, nextTimestamp()), - new Event(5, nextTimestamp()), - new Event(6, nextTimestamp()) - }; + Event[] set1 = + new Event[] { + new Event(1, nextTimestamp()), new Event(2, nextTimestamp()), + }; + Event[] set2 = + new Event[] { + new Event(4, nextTimestamp()), + new Event(5, nextTimestamp()), + new Event(6, nextTimestamp()) + }; ContiguousSequenceRange expectedResult = ContiguousSequenceRange.EMPTY; int expectedNumberOfRanges = 2; @@ -233,19 +257,20 @@ public void testMergingWithoutAnyInitialEvents() { @Test public void testMergingAdjacentRanges() { - Event[] set1 = new Event[]{ - new Event(1, nextTimestamp(), true), - new Event(2, nextTimestamp()), - }; - Event[] set2 = new Event[]{ - new Event(3, nextTimestamp()), - new Event(4, nextTimestamp()), - new Event(5, nextTimestamp()), - new Event(6, nextTimestamp()) - }; - - ContiguousSequenceRange expectedResult = ContiguousSequenceRange.of(1, 6, - eventTimestamp(set2, 6L)); + Event[] set1 = + new Event[] { + new Event(1, nextTimestamp(), true), new Event(2, nextTimestamp()), + }; + Event[] set2 = + new Event[] { + new Event(3, nextTimestamp()), + new Event(4, nextTimestamp()), + new Event(5, nextTimestamp()), + new Event(6, nextTimestamp()) + }; + + ContiguousSequenceRange expectedResult = + ContiguousSequenceRange.of(1, 6, eventTimestamp(set2, 6L)); int expectedNumberOfRanges = 1; doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); @@ -253,19 +278,20 @@ public void testMergingAdjacentRanges() { @Test public void testPruningSequencesBelowInitial() { - Event[] set1 = new Event[]{ - new Event(1, nextTimestamp()), - new Event(2, nextTimestamp()), - }; - Event[] set2 = new Event[]{ - new Event(3, nextTimestamp(), true), - new Event(4, nextTimestamp()), - new Event(5, nextTimestamp()), - new Event(6, nextTimestamp()) - }; - - ContiguousSequenceRange expectedResult = ContiguousSequenceRange.of(3, 6, - eventTimestamp(set2, 6L)); + Event[] set1 = + new Event[] { + new Event(1, nextTimestamp()), new Event(2, nextTimestamp()), + }; + Event[] set2 = + new Event[] { + new Event(3, nextTimestamp(), true), + new Event(4, nextTimestamp()), + new Event(5, nextTimestamp()), + new Event(6, nextTimestamp()) + }; + + ContiguousSequenceRange expectedResult = + ContiguousSequenceRange.of(3, 6, eventTimestamp(set2, 6L)); int expectedNumberOfRanges = 1; doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); @@ -273,21 +299,23 @@ public void testPruningSequencesBelowInitial() { @Test public void testDuplicateHandling() { - Event[] set1 = new Event[]{ - new Event(1, nextTimestamp(), true), - new Event(2, nextTimestamp()), - new Event(3, nextTimestamp()), - new Event(5, nextTimestamp()), - }; - Event[] set2 = new Event[]{ - new Event(3, nextTimestamp()), - new Event(4, nextTimestamp()), - new Event(5, nextTimestamp()), - new Event(6, nextTimestamp()) - }; - - ContiguousSequenceRange expectedResult = ContiguousSequenceRange.of(1, 6, - eventTimestamp(set2, 6L)); + Event[] set1 = + new Event[] { + new Event(1, nextTimestamp(), true), + new Event(2, nextTimestamp()), + new Event(3, nextTimestamp()), + new Event(5, nextTimestamp()), + }; + Event[] set2 = + new Event[] { + new Event(3, nextTimestamp()), + new Event(4, nextTimestamp()), + new Event(5, nextTimestamp()), + new Event(6, nextTimestamp()) + }; + + ContiguousSequenceRange expectedResult = + ContiguousSequenceRange.of(1, 6, eventTimestamp(set2, 6L)); int expectedNumberOfRanges = 1; doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); @@ -295,47 +323,53 @@ public void testDuplicateHandling() { @Test public void testExceptionThrownIfThereAreDifferentInitialSequences() { - Event[] set1 = new Event[]{ - new Event(1, nextTimestamp(), true), - new Event(2, nextTimestamp()), - }; - Event[] set2 = new Event[]{ - new Event(3, nextTimestamp(), true), - new Event(4, nextTimestamp()), - new Event(5, nextTimestamp()), - new Event(6, nextTimestamp()) - }; + Event[] set1 = + new Event[] { + new Event(1, nextTimestamp(), true), new Event(2, nextTimestamp()), + }; + Event[] set2 = + new Event[] { + new Event(3, nextTimestamp(), true), + new Event(4, nextTimestamp()), + new Event(5, nextTimestamp()), + new Event(6, nextTimestamp()) + }; try { doTestMerging(set1, set2, ContiguousSequenceRange.EMPTY, 0); Assert.fail("Expected to throw an exception"); } catch (IllegalStateException e) { - Assert.assertEquals("Exception message", - "Two accumulators contain different initial sequences: 1 and 3", e.getMessage()); + Assert.assertEquals( + "Exception message", + "Two accumulators contain different initial sequences: 1 and 3", + e.getMessage()); } } - @Test public void testSelectingHighestTimestampWhenMerging() { - Event[] set1 = new Event[]{ - new Event(1, nextTimestamp(), true), - new Event(2, Instant.ofEpochMilli(currentTicker.get() + 10000)), - }; - Event[] set2 = new Event[]{ - new Event(3, nextTimestamp()), - new Event(4, nextTimestamp()), - new Event(5, nextTimestamp()), - new Event(6, nextTimestamp()) - }; - - ContiguousSequenceRange expectedResult = ContiguousSequenceRange.of(1, 6, - eventTimestamp(set1, 2L)); + Event[] set1 = + new Event[] { + new Event(1, nextTimestamp(), true), + new Event(2, Instant.ofEpochMilli(currentTicker.get() + 10000)), + }; + Event[] set2 = + new Event[] { + new Event(3, nextTimestamp()), + new Event(4, nextTimestamp()), + new Event(5, nextTimestamp()), + new Event(6, nextTimestamp()) + }; + + ContiguousSequenceRange expectedResult = + ContiguousSequenceRange.of(1, 6, eventTimestamp(set1, 2L)); int expectedNumberOfRanges = 1; doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); } - private static void doTestMerging(Event[] set1, Event[] set2, + private static void doTestMerging( + Event[] set1, + Event[] set2, ContiguousSequenceRange expectedResult, int expectedNumberOfRanges) { // Try to merge both set2 to set1 and set1 to set2 - both must return the same results @@ -343,9 +377,12 @@ private static void doTestMerging(Event[] set1, Event[] set2, mergeAndTest(set2, set1, expectedResult, expectedNumberOfRanges, "set2"); } - private static void mergeAndTest(Event[] set1, Event[] set2, + private static void mergeAndTest( + Event[] set1, + Event[] set2, ContiguousSequenceRange expectedResult, - int expectedNumberOfRanges, String firstSetName) { + int expectedNumberOfRanges, + String firstSetName) { final SequenceRangeAccumulator a1 = new SequenceRangeAccumulator(); Arrays.stream(set1).forEach(e -> a1.add(e.sequence, e.timestamp, e.initialEvent)); @@ -354,12 +391,10 @@ private static void mergeAndTest(Event[] set1, Event[] set2, a1.merge(a2); - Assert.assertEquals("Accumulated results - " + firstSetName, - expectedResult, - a1.largestContinuousRange()); + Assert.assertEquals( + "Accumulated results - " + firstSetName, expectedResult, a1.largestContinuousRange()); - Assert.assertEquals("Number of ranges - " + firstSetName, expectedNumberOfRanges, - a1.numberOfRanges()); + Assert.assertEquals( + "Number of ranges - " + firstSetName, expectedNumberOfRanges, a1.numberOfRanges()); } - } From b73a4d6e88b4ba9aaf5bd93989e44ae100853fe3 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Mon, 23 Sep 2024 16:03:50 -0700 Subject: [PATCH 21/33] Polish by "spotless" --- .../ordered/GlobalSequenceTracker.java | 2 +- .../ordered/combiner/package-info.java | 23 +++++++++++++++++++ 2 files changed, 24 insertions(+), 1 deletion(-) create mode 100644 sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/package-info.java diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java index 671ca5dd4873..ccdf410ba996 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java @@ -34,7 +34,7 @@ import org.joda.time.Duration; /** - * PTransform to produce the side input of the maximum contiguous range of sequence numbers + * PTransform to produce the side input of the maximum contiguous range of sequence numbers. * * @param type of event key * @param type of event diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/package-info.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/package-info.java new file mode 100644 index 000000000000..0d730d55fb9f --- /dev/null +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/package-info.java @@ -0,0 +1,23 @@ +/* + * 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. + */ +/** + * Default implementation of the global sequence combiner used by {@link + * org.apache.beam.sdk.extensions.ordered.OrderedEventProcessor} when processing events using global + * sequences. + */ +package org.apache.beam.sdk.extensions.ordered.combiner; From 3035e5a8fa05ca2c22eb7097226a9312f3bd6481 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Mon, 23 Sep 2024 16:10:02 -0700 Subject: [PATCH 22/33] Removed unneeded logging configuration file. --- .../ordered/src/test/resources/logging.properties | 15 --------------- 1 file changed, 15 deletions(-) delete mode 100644 sdks/java/extensions/ordered/src/test/resources/logging.properties diff --git a/sdks/java/extensions/ordered/src/test/resources/logging.properties b/sdks/java/extensions/ordered/src/test/resources/logging.properties deleted file mode 100644 index 64edb0a5fa9b..000000000000 --- a/sdks/java/extensions/ordered/src/test/resources/logging.properties +++ /dev/null @@ -1,15 +0,0 @@ -handlers = java.util.logging.ConsoleHandler -java.util.logging.ConsoleHandler.level = ALL -java.util.logging.ConsoleHandler.formatter = java.util.logging.SimpleFormatter -# Pattern works since Java 7 -java.util.logging.SimpleFormatter.format = [%1$tc] %4$s: %2$s - %5$s %6$s%n - -# Configure logging levels -# Available log levels are: -# OFF, SEVERE, WARNING, INFO, CONFIG, FINE, FINER, FINEST, ALL - -# root logger -.level = FINEST - -# child logger -org.example.level = ALL \ No newline at end of file From a1874b11f066f9b1b9ecd297c04c899c02d72b41 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Tue, 8 Oct 2024 11:55:39 -0700 Subject: [PATCH 23/33] Made ContiguousSequenceRange open ended. --- .../ordered/ContiguousSequenceRange.java | 2 +- .../extensions/ordered/ProcessingState.java | 2 +- .../sdk/extensions/ordered/ProcessorDoFn.java | 2 +- .../combiner/SequenceRangeAccumulator.java | 10 +++++++- .../SequenceRangeAccumulatorTest.java | 24 +++++++++---------- 5 files changed, 24 insertions(+), 16 deletions(-) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ContiguousSequenceRange.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ContiguousSequenceRange.java index 7bfebcb6a0de..c16cf9328dcd 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ContiguousSequenceRange.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ContiguousSequenceRange.java @@ -40,7 +40,7 @@ public abstract class ContiguousSequenceRange { /** @return inclusive starting sequence */ public abstract long getStart(); - /** @return inclusive end sequence */ + /** @return exclusive end sequence */ public abstract long getEnd(); /** @return latest timestamp of all events in the range */ diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java index fcd1b4b604ec..425eb4444a63 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java @@ -326,7 +326,7 @@ public boolean thereAreGloballySequencedEventsToBeProcessed() { return bufferedEventCount > 0 && lastCompleteGlobalSequence != null && earliestBufferedSequence != null - && earliestBufferedSequence <= lastCompleteGlobalSequence.getEnd(); + && earliestBufferedSequence < lastCompleteGlobalSequence.getEnd(); } /** diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java index 1cfceab61cce..38fb1d6454ff 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java @@ -359,7 +359,7 @@ StateT processBufferedEventRange( boolean continueProcessing = checkForSequenceGapInBufferedEvents() ? currentEventIsNextInSequence - : (eventSequence <= contiguousSequenceRange.getEnd() || currentEventIsNextInSequence); + : (eventSequence < contiguousSequenceRange.getEnd() || currentEventIsNextInSequence); if (!continueProcessing) { processingState.foundSequenceGap(eventSequence); // Records will be cleared up to this element diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java index 1c627ea7980f..a9063d61bf59 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java @@ -56,6 +56,12 @@ public void add(long sequence, Instant timestamp, boolean isInitialSequence) { + sequence); } + if(sequence == Long.MAX_VALUE) { + // This is an invalid value and DoFns will not process this element. This will also allow + // to produce a ContiguousSequenceRange with the exclusive end value. + return; + } + if (isInitialSequence) { this.initialSequence = sequence; clearRangesBelowInitialSequence(sequence, timestamp); @@ -122,7 +128,9 @@ public ContiguousSequenceRange largestContinuousRange() { Long start = firstEntry.getKey(); Long end = firstEntry.getValue().getLeft(); Instant latestTimestamp = firstEntry.getValue().getRight(); - return ContiguousSequenceRange.of(start, end, latestTimestamp); + // Upper bound is inclusive, but the ContiguousSequenceRange's end is exclusive. + // The numeric overflow is prevented by dropping the value of Long.MAX. + return ContiguousSequenceRange.of(start, end + 1, latestTimestamp); } public int numberOfRanges() { diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java index e92a2ef517b6..4082ce6de758 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java @@ -68,7 +68,7 @@ public void testSimpleAccumulation() { new Event(3, nextTimestamp()) }; - doTestAccumulation(events, ContiguousSequenceRange.of(1, 3, eventTimestamp(events, 3)), 1); + doTestAccumulation(events, ContiguousSequenceRange.of(1, 4, eventTimestamp(events, 3)), 1); } @Test @@ -81,7 +81,7 @@ public void testReverseArrivalHandling() { }; Instant timestampOfEventNumber1 = eventTimestamp(events, 1); - doTestAccumulation(events, ContiguousSequenceRange.of(1, 3, timestampOfEventNumber1), 1); + doTestAccumulation(events, ContiguousSequenceRange.of(1, 4, timestampOfEventNumber1), 1); } @Test @@ -95,7 +95,7 @@ public void testPartialRangeAccumulation() { new Event(7, nextTimestamp()), }; - doTestAccumulation(events, ContiguousSequenceRange.of(1, 3, eventTimestamp(events, 3)), 3); + doTestAccumulation(events, ContiguousSequenceRange.of(1, 4, eventTimestamp(events, 3)), 3); } @Test @@ -110,7 +110,7 @@ public void testMergingRangeAccumulation() { new Event(6, nextTimestamp()), }; - doTestAccumulation(events, ContiguousSequenceRange.of(1, 3, eventTimestamp(events, 3)), 2); + doTestAccumulation(events, ContiguousSequenceRange.of(1, 4, eventTimestamp(events, 3)), 2); } @Test @@ -150,7 +150,7 @@ public void testRemovingRangesBelowInitialSequenceDuringAccumulation() { new Event(7, nextTimestamp(), true), }; - doTestAccumulation(events, ContiguousSequenceRange.of(7, 7, eventTimestamp(events, 7)), 1); + doTestAccumulation(events, ContiguousSequenceRange.of(7, 8, eventTimestamp(events, 7)), 1); } @Test @@ -174,7 +174,7 @@ public void testRemovingElementsBelowInitialSequenceDuringAccumulation() { }; Instant timestampOfTheLastEvent = events[events.length - 1].timestamp; - doTestAccumulation(events, ContiguousSequenceRange.of(7, 8, timestampOfTheLastEvent), 1); + doTestAccumulation(events, ContiguousSequenceRange.of(7, 9, timestampOfTheLastEvent), 1); } private static void doTestAccumulation( @@ -210,7 +210,7 @@ public void testMergingNonEmptyWithEmpty() { Event[] set2 = new Event[] {}; ContiguousSequenceRange expectedResult = - ContiguousSequenceRange.of(1, 3, eventTimestamp(set1, 1L)); + ContiguousSequenceRange.of(1, 4, eventTimestamp(set1, 1L)); int expectedNumberOfRanges = 1; doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); @@ -230,7 +230,7 @@ public void testMergingWithLowerNonAdjacentRange() { }; ContiguousSequenceRange expectedResult = - ContiguousSequenceRange.of(1, 2, eventTimestamp(set1, 2L)); + ContiguousSequenceRange.of(1, 3, eventTimestamp(set1, 2L)); int expectedNumberOfRanges = 2; doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); @@ -270,7 +270,7 @@ public void testMergingAdjacentRanges() { }; ContiguousSequenceRange expectedResult = - ContiguousSequenceRange.of(1, 6, eventTimestamp(set2, 6L)); + ContiguousSequenceRange.of(1, 7, eventTimestamp(set2, 6L)); int expectedNumberOfRanges = 1; doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); @@ -291,7 +291,7 @@ public void testPruningSequencesBelowInitial() { }; ContiguousSequenceRange expectedResult = - ContiguousSequenceRange.of(3, 6, eventTimestamp(set2, 6L)); + ContiguousSequenceRange.of(3, 7, eventTimestamp(set2, 6L)); int expectedNumberOfRanges = 1; doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); @@ -315,7 +315,7 @@ public void testDuplicateHandling() { }; ContiguousSequenceRange expectedResult = - ContiguousSequenceRange.of(1, 6, eventTimestamp(set2, 6L)); + ContiguousSequenceRange.of(1, 7, eventTimestamp(set2, 6L)); int expectedNumberOfRanges = 1; doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); @@ -362,7 +362,7 @@ public void testSelectingHighestTimestampWhenMerging() { }; ContiguousSequenceRange expectedResult = - ContiguousSequenceRange.of(1, 6, eventTimestamp(set1, 2L)); + ContiguousSequenceRange.of(1, 7, eventTimestamp(set1, 2L)); int expectedNumberOfRanges = 1; doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); } From 3a8e3a1b1cd8a0931f69f748c803aff9bd141a30 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Tue, 8 Oct 2024 11:59:24 -0700 Subject: [PATCH 24/33] Removed details from 2.60.0 section in CHANGES.md. --- CHANGES.md | 1 - 1 file changed, 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index cb97c546da65..d58ceffeb411 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -68,7 +68,6 @@ * Dataflow worker can install packages from Google Artifact Registry Python repositories (Python) ([#32123](https://github.com/apache/beam/issues/32123)). * Added support for Zstd codec in SerializableAvroCodecFactory (Java) ([#32349](https://github.com/apache/beam/issues/32349)) -* Added support for processing events which use a global sequence to "ordered" extension (Java) * X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). ## Breaking Changes From 69a1eaed29394e4fff4fd041c78903034ea93e7c Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Tue, 8 Oct 2024 12:02:42 -0700 Subject: [PATCH 25/33] Update sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java Co-authored-by: Danny McCormick --- .../extensions/ordered/combiner/DefaultSequenceCombiner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java index f26964670799..498c8621c09c 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java @@ -40,7 +40,7 @@ /** * Default global sequence combiner. * - *

    Produces {@link ContiguousSequenceRange} of contiguous longs starting from the initial event + *

    Produces the largest {@link ContiguousSequenceRange} of contiguous longs which starts from the initial event * identified by {@link EventExaminer#isInitialEvent(long, EventT)}. * *

    This combiner currently doesn't use {@link EventExaminer#isLastEvent(long, EventT)}. From 76d769c2b2289a407d0643978362b4eb334c504b Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Tue, 8 Oct 2024 12:26:28 -0700 Subject: [PATCH 26/33] Fixed spotless related errors. --- .../extensions/ordered/combiner/DefaultSequenceCombiner.java | 4 ++-- .../extensions/ordered/combiner/SequenceRangeAccumulator.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java index 498c8621c09c..32e5cbc36e4e 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java @@ -40,8 +40,8 @@ /** * Default global sequence combiner. * - *

    Produces the largest {@link ContiguousSequenceRange} of contiguous longs which starts from the initial event - * identified by {@link EventExaminer#isInitialEvent(long, EventT)}. + *

    Produces the largest {@link ContiguousSequenceRange} of contiguous longs which starts from the + * initial event identified by {@link EventExaminer#isInitialEvent(long, EventT)}. * *

    This combiner currently doesn't use {@link EventExaminer#isLastEvent(long, EventT)}. * diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java index a9063d61bf59..67776b63503b 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java @@ -56,7 +56,7 @@ public void add(long sequence, Instant timestamp, boolean isInitialSequence) { + sequence); } - if(sequence == Long.MAX_VALUE) { + if (sequence == Long.MAX_VALUE) { // This is an invalid value and DoFns will not process this element. This will also allow // to produce a ContiguousSequenceRange with the exclusive end value. return; From 7d80980b9475c450a2263f20c7b74ff2e3e8345c Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Tue, 8 Oct 2024 14:17:56 -0700 Subject: [PATCH 27/33] Added a note about the new functionality to CHANGES.md --- CHANGES.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGES.md b/CHANGES.md index 364b1a5fbdef..ca54f5cb4834 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -67,6 +67,7 @@ ## New Features / Improvements * X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* Added support for processing events which use a global sequence to "ordered" extension (Java) [#32540](https://github.com/apache/beam/pull/32540) ## Breaking Changes From 7a3b7fa9a51908736407210616507f22044a8e80 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Tue, 8 Oct 2024 14:34:03 -0700 Subject: [PATCH 28/33] Added clarification around the data structure used in the sequence combiner. --- .../ordered/combiner/SequenceRangeAccumulator.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java index 67776b63503b..b26397ccae77 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java @@ -44,6 +44,16 @@ private static Instant max(Instant a, Instant b) { return a.isAfter(b) ? a : b; } + + /** + * The tree contains a set of non-overlapping contiguous ranges, where the key is the lower + * inclusive start of the range, left value of the pair is the inclusive end of the range and + * the right value of the pair is the maximum timestamp in the range. + * + * The maximum timestamp is critical for the correctness of the ordered processing. + * During the merge process the merged range is assigned the maximum timestamp of + * the two ranges that created this new range. + */ private final TreeMap> data = new TreeMap<>(); private @Nullable Long initialSequence = null; From 81acfcbb6820f823a36661559f211560d496cd26 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Tue, 8 Oct 2024 14:34:40 -0700 Subject: [PATCH 29/33] Added clarification around the data structure used in the sequence combiner. --- .../ordered/combiner/SequenceRangeAccumulator.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java index b26397ccae77..89dc912afc90 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java @@ -44,17 +44,17 @@ private static Instant max(Instant a, Instant b) { return a.isAfter(b) ? a : b; } - /** * The tree contains a set of non-overlapping contiguous ranges, where the key is the lower - * inclusive start of the range, left value of the pair is the inclusive end of the range and - * the right value of the pair is the maximum timestamp in the range. + * inclusive start of the range, left value of the pair is the inclusive end of the range and the + * right value of the pair is the maximum timestamp in the range. * - * The maximum timestamp is critical for the correctness of the ordered processing. - * During the merge process the merged range is assigned the maximum timestamp of - * the two ranges that created this new range. + *

    The maximum timestamp is critical for the correctness of the ordered processing. During the + * merge process the merged range is assigned the maximum timestamp of the two ranges that created + * this new range. */ private final TreeMap> data = new TreeMap<>(); + private @Nullable Long initialSequence = null; public void add(long sequence, Instant timestamp, boolean isInitialSequence) { From f54bdda60c747e635a7e36e3c200a5b598b4f301 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Tue, 8 Oct 2024 15:12:51 -0700 Subject: [PATCH 30/33] Fixed the problem with allowed lateness being set to 0 in the global sequence tracker. --- .../beam/sdk/extensions/ordered/GlobalSequenceTracker.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java index ccdf410ba996..9b42fc42d2ad 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java @@ -89,8 +89,7 @@ public PCollectionView expand( (WindowFn>>, ?>) input.getWindowingStrategy().getWindowFn()) .accumulatingFiredPanes() - // TODO: verify that we don't need to have the lateness parameterized - .withAllowedLateness(Duration.ZERO) + .withAllowedLateness(input.getWindowingStrategy().getAllowedLateness()) .triggering( Repeatedly.forever( AfterFirst.of( From 087dee638c29cbc5c715890c3fd7fee905974b72 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Tue, 8 Oct 2024 16:32:44 -0700 Subject: [PATCH 31/33] Parameterized the GlobalSequenceTracker with the max number of events to trigger the re-evaluation. Fixed accidentally disabled unit tests. --- .../ordered/GlobalSequenceTracker.java | 14 +++++- .../ordered/OrderedEventProcessor.java | 6 ++- .../ordered/OrderedProcessingHandler.java | 29 +++++++++--- ...deredEventProcessorGlobalSequenceTest.java | 46 +++++++++---------- 4 files changed, 61 insertions(+), 34 deletions(-) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java index 9b42fc42d2ad..aa12c30a5317 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java @@ -51,22 +51,31 @@ class GlobalSequenceTracker< TimestampedValue>>, ContiguousSequenceRange> sideInputProducer; private final @Nullable Duration frequencyOfGeneration; + private final int maxElementsBeforeReevaluatingGlobalSequence; + /** + * Constructor used in batch pipelines. + * + * @param sideInputProducer + */ public GlobalSequenceTracker( Combine.GloballyAsSingletonView< TimestampedValue>>, ContiguousSequenceRange> sideInputProducer) { this.sideInputProducer = sideInputProducer; this.frequencyOfGeneration = null; + this.maxElementsBeforeReevaluatingGlobalSequence = 0; } public GlobalSequenceTracker( Combine.GloballyAsSingletonView< TimestampedValue>>, ContiguousSequenceRange> sideInputProducer, - Duration globalSequenceGenerationFrequency) { + Duration globalSequenceGenerationFrequency, + int maxElementsBeforeReevaluatingGlobalSequence) { this.sideInputProducer = sideInputProducer; this.frequencyOfGeneration = globalSequenceGenerationFrequency; + this.maxElementsBeforeReevaluatingGlobalSequence = maxElementsBeforeReevaluatingGlobalSequence; } @Override @@ -93,7 +102,8 @@ public PCollectionView expand( .triggering( Repeatedly.forever( AfterFirst.of( - AfterPane.elementCountAtLeast(1), + AfterPane.elementCountAtLeast( + maxElementsBeforeReevaluatingGlobalSequence), AfterProcessingTime.pastFirstElementInPane() .plusDelayOf(frequencyOfGeneration))))); } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java index 913c3be7aea2..7eb1a4a326b7 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java @@ -243,7 +243,9 @@ private OrderedEventProcessorResult expandGlobalSequ streamingProcessing ? new GlobalSequenceTracker<>( globalSequenceHandler.getGlobalSequenceCombiner(), - globalSequenceHandler.getGlobalSequenceGenerationFrequency()) + globalSequenceHandler.getContiguousSequenceRangeReevaluationFrequency(), + globalSequenceHandler + .getMaxElementCountToTriggerContinuousSequenceRangeReevaluation()) : new GlobalSequenceTracker<>( globalSequenceHandler.getGlobalSequenceCombiner())); @@ -254,7 +256,7 @@ private OrderedEventProcessorResult expandGlobalSequ new PerKeyTickerGenerator<>( keyCoder, eventCoder, - globalSequenceHandler.getFrequencyOfCheckingForNewGlobalSequence())); + globalSequenceHandler.getContiguousSequenceRangeReevaluationFrequency())); input = PCollectionList.of(input) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java index 4844f12bd961..d8ad13330a1a 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedProcessingHandler.java @@ -261,8 +261,13 @@ public OrderedProcessingGlobalSequenceHandler( } /** - * How frequently the combiner should generate a new sequence? This parameter only affects the - * behaviour of streaming pipelines. + * How frequently the combiner should reevaluate the maximum range? This parameter only affects + * the behaviour of streaming pipelines. + * + *

    This parameter is used together with {@link + * OrderedProcessingGlobalSequenceHandler#getMaxElementCountToTriggerContinuousSequenceRangeReevaluation()}. + * The re-evaluation will occur as soon as the number of new elements exceeds the threshold or + * the time exceeds the frequency. * *

    Notice that some runners cache the output of side inputs and this parameter might not * appear to have an effect unless the cache time-to-live is equal or less than this frequency. @@ -270,14 +275,26 @@ public OrderedProcessingGlobalSequenceHandler( * href="https://beam.apache.org/releases/javadoc/current/org/apache/beam/runners/dataflow/options/DataflowStreamingPipelineOptions.html#getStreamingSideInputCacheExpirationMillis--">this * Dataflow streaming pipeline option} * - * @return frequency of generating new global sequence. Default - every second. + * @return frequency of reevaluating the {@link ContiguousSequenceRange}. Default - every + * second. + * @see + * OrderedProcessingGlobalSequenceHandler#getMaxElementCountToTriggerContinuousSequenceRangeReevaluation() */ - public Duration getFrequencyOfCheckingForNewGlobalSequence() { + public Duration getContiguousSequenceRangeReevaluationFrequency() { return Duration.standardSeconds(1); } - public Duration getGlobalSequenceGenerationFrequency() { - return Duration.standardSeconds(1); + /** + * Number of new elements to trigger the re-evaluation. + * + *

    See {@link + * OrderedProcessingGlobalSequenceHandler#getContiguousSequenceRangeReevaluationFrequency()} for + * additional details. + * + * @return batch size. Default - 1000. + */ + public int getMaxElementCountToTriggerContinuousSequenceRangeReevaluation() { + return 1000; } } } diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java index d69d41601868..0763a8fe72ca 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java @@ -74,7 +74,7 @@ public void testPerfectOrderingProcessing() throws CannotProvideCoderException { EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, LARGE_MAX_RESULTS_PER_OUTPUT, - ContiguousSequenceRange.of(0, 5, new Instant())); + ContiguousSequenceRange.of(0, 6, new Instant())); } @Test @@ -108,7 +108,7 @@ public void testOutOfSequenceProcessing() throws CannotProvideCoderException { EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, LARGE_MAX_RESULTS_PER_OUTPUT, - ContiguousSequenceRange.of(0, 8, new Instant())); + ContiguousSequenceRange.of(0, 9, new Instant())); } @Test @@ -147,7 +147,7 @@ public void testHandlingOfDuplicateSequences() throws CannotProvideCoderExceptio EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, LARGE_MAX_RESULTS_PER_OUTPUT, - ContiguousSequenceRange.of(0, 3, new Instant())); + ContiguousSequenceRange.of(0, 4, new Instant())); } @Test @@ -180,7 +180,7 @@ public void testTreatingSequencesBelowInitialAsDuplicates() throws CannotProvide EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, LARGE_MAX_RESULTS_PER_OUTPUT, - ContiguousSequenceRange.of(0, 3, new Instant())); + ContiguousSequenceRange.of(0, 4, new Instant())); } @Test @@ -218,7 +218,7 @@ public void testHandlingOfCheckedExceptions() throws CannotProvideCoderException // Sequence matcher doesn't know if the element is valid or not. // That's why the elements that are get rejected in the processor still count when // calculating the global sequence - ContiguousSequenceRange.of(0, 3, new Instant())); + ContiguousSequenceRange.of(0, 4, new Instant())); } @Test @@ -245,7 +245,7 @@ public void testProcessingWithEveryOtherResultEmission() throws CannotProvideCod EMISSION_FREQUENCY_ON_EVERY_OTHER_EVENT, INITIAL_SEQUENCE_OF_0, LARGE_MAX_RESULTS_PER_OUTPUT, - ContiguousSequenceRange.of(0, 5, new Instant())); + ContiguousSequenceRange.of(0, 6, new Instant())); } @Test @@ -279,7 +279,7 @@ public void testLargeBufferedOutputInTimer() throws CannotProvideCoderException EMISSION_FREQUENCY_ON_EVERY_ELEMENT, 1L /* This dataset assumes 1 as the starting sequence */, maxResultsPerOutput, - ContiguousSequenceRange.of(1, sequences.length, new Instant())); + ContiguousSequenceRange.of(1, sequences.length + 1, new Instant())); } @Test @@ -312,7 +312,7 @@ public void testSequenceGapProcessingInBufferedOutput() throws CannotProvideCode EMISSION_FREQUENCY_ON_EVERY_ELEMENT, 1L /* This dataset assumes 1 as the starting sequence */, maxResultsPerOutput, - ContiguousSequenceRange.of(1, 10, new Instant())); + ContiguousSequenceRange.of(1, 11, new Instant())); } @Test @@ -345,7 +345,7 @@ public void testHandlingOfMaxSequenceNumber() throws CannotProvideCoderException EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, LARGE_MAX_RESULTS_PER_OUTPUT, - ContiguousSequenceRange.of(0, 2, Instant.now())); + ContiguousSequenceRange.of(0, 3, Instant.now())); } @Test @@ -368,7 +368,7 @@ public void testProcessingOfTheLastInput() throws CannotProvideCoderException { EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, LARGE_MAX_RESULTS_PER_OUTPUT, - ContiguousSequenceRange.of(0, 2, new Instant())); + ContiguousSequenceRange.of(0, 3, new Instant())); } private void testGlobalSequenceProcessing( @@ -399,20 +399,18 @@ private void testGlobalSequenceProcessing( ContiguousSequenceRange expectedLastCompleteRange) throws CannotProvideCoderException { // Test a streaming pipeline - if (false) { - doTest( - events, - null /* expectedStatuses */, - expectedOutput, - expectedUnprocessedEvents, - emissionFrequency, - initialSequence, - maxResultsPerOutput, - false /* produceStatusOnEveryEvent */, - STREAMING, - GLOBAL_SEQUENCE, - expectedLastCompleteRange); - } + doTest( + events, + null /* expectedStatuses */, + expectedOutput, + expectedUnprocessedEvents, + emissionFrequency, + initialSequence, + maxResultsPerOutput, + false /* produceStatusOnEveryEvent */, + STREAMING, + GLOBAL_SEQUENCE, + expectedLastCompleteRange); // Test a batch pipeline if (runTestsOnDataflowRunner()) { From 479e60acf993c20a1b0995d4ed3979ebf1bdbed6 Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Tue, 8 Oct 2024 16:45:15 -0700 Subject: [PATCH 32/33] Made the event timer used to wait for the event arrival respect the lateness of the input. --- .../extensions/ordered/GlobalSequencesProcessorDoFn.java | 8 ++++++-- .../sdk/extensions/ordered/OrderedEventProcessor.java | 3 ++- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java index a3d84c6109a9..64c2d119c97d 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java @@ -81,6 +81,8 @@ class GlobalSequencesProcessorDoFn< private final PCollectionView latestContiguousRangeSideInput; + private final Duration maxLateness; + GlobalSequencesProcessorDoFn( EventExaminer eventExaminer, Coder eventCoder, @@ -92,7 +94,8 @@ class GlobalSequencesProcessorDoFn< TupleTag>>> unprocessedEventTupleTag, boolean produceStatusUpdateOnEveryEvent, long maxNumberOfResultsToProduce, - PCollectionView latestContiguousRangeSideInput) { + PCollectionView latestContiguousRangeSideInput, + Duration maxLateness) { super( eventExaminer, mainOutputTupleTag, @@ -107,6 +110,7 @@ class GlobalSequencesProcessorDoFn< this.processingStateSpec = StateSpecs.value(ProcessingStateCoder.of(keyCoder)); this.mutableStateSpec = StateSpecs.value(stateCoder); this.windowClosedSpec = StateSpecs.value(BooleanCoder.of()); + this.maxLateness = maxLateness; } @Override @@ -197,7 +201,7 @@ private void setBatchEmissionTimerIfNeeded( ContiguousSequenceRange lastCompleteGlobalSequence = processingState.getLastContiguousRange(); if (lastCompleteGlobalSequence != null && processingState.thereAreGloballySequencedEventsToBeProcessed()) { - batchEmissionTimer.set(lastCompleteGlobalSequence.getTimestamp()); + batchEmissionTimer.set(lastCompleteGlobalSequence.getTimestamp().plus(maxLateness)); } } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java index 7eb1a4a326b7..fb23a7c8667a 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessor.java @@ -278,7 +278,8 @@ private OrderedEventProcessorResult expandGlobalSequ unprocessedEventOutput, handler.isProduceStatusUpdateOnEveryEvent(), handler.getMaxOutputElementsPerBundle(), - latestContiguousRange)) + latestContiguousRange, + input.getWindowingStrategy().getAllowedLateness())) .withOutputTags( mainOutput, TupleTagList.of(Arrays.asList(statusOutput, unprocessedEventOutput))) From f11dabf72c85ac4d676cd9894a69cf836065ca1f Mon Sep 17 00:00:00 2001 From: Sergei Lilichenko Date: Tue, 8 Oct 2024 16:52:52 -0700 Subject: [PATCH 33/33] Created new failure reason code - "before initial sequence" --- .../beam/sdk/extensions/ordered/ProcessorDoFn.java | 9 ++++++++- .../beam/sdk/extensions/ordered/UnprocessedEvent.java | 4 ++-- .../OrderedEventProcessorGlobalSequenceTest.java | 10 ++++++++-- 3 files changed, 18 insertions(+), 5 deletions(-) diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java index 38fb1d6454ff..a05b0829074a 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java @@ -327,10 +327,12 @@ StateT processBufferedEventRange( EventT bufferedEvent = timestampedEvent.getValue(); boolean skipProcessing = false; + boolean beforeInitialSequence = false; if (contiguousSequenceRange != null && eventSequence < contiguousSequenceRange.getStart()) { // In case of global sequence processing - remove the elements below the range start skipProcessing = true; + beforeInitialSequence = true; endClearRange = fromLong(eventSequence); } if (processingState.checkForDuplicateBatchedEvent(eventSequence)) { @@ -346,7 +348,12 @@ StateT processBufferedEventRange( KV.of( processingState.getKey(), KV.of( - eventSequence, UnprocessedEvent.create(bufferedEvent, Reason.duplicate)))); + eventSequence, + UnprocessedEvent.create( + bufferedEvent, + beforeInitialSequence + ? Reason.before_initial_sequence + : Reason.duplicate)))); // TODO: When there is a large number of duplicates this can cause a situation where // we produce too much output and the runner will start throwing unrecoverable errors. // Need to add counting logic to accumulate both the normal and DLQ outputs. diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/UnprocessedEvent.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/UnprocessedEvent.java index 3c8f655ffca6..d7c599277567 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/UnprocessedEvent.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/UnprocessedEvent.java @@ -68,12 +68,12 @@ static UnprocessedEvent create( return new AutoValue_UnprocessedEvent<>(event, reason, failureDetails); } - // TODO: do we need another reason, "before the initial event"? public enum Reason { duplicate, buffered, sequence_id_outside_valid_range, - exception_thrown + exception_thrown, + before_initial_sequence }; public abstract EventT getEvent(); diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java index 0763a8fe72ca..98bc7591f4d7 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java @@ -170,8 +170,14 @@ public void testTreatingSequencesBelowInitialAsDuplicates() throws CannotProvide expectedOutput.add(KV.of("id-1", "abcd")); Collection>>> duplicates = new ArrayList<>(); - duplicates.add(KV.of("id-1", KV.of(-1L, UnprocessedEvent.create("early-1", Reason.duplicate)))); - duplicates.add(KV.of("id-1", KV.of(-2L, UnprocessedEvent.create("early-2", Reason.duplicate)))); + duplicates.add( + KV.of( + "id-1", + KV.of(-1L, UnprocessedEvent.create("early-1", Reason.before_initial_sequence)))); + duplicates.add( + KV.of( + "id-1", + KV.of(-2L, UnprocessedEvent.create("early-2", Reason.before_initial_sequence)))); testGlobalSequenceProcessing( events,