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 diff --git a/sdks/java/extensions/ordered/build.gradle b/sdks/java/extensions/ordered/build.gradle index 10c9785b9eed..8bee1901bd3a 100644 --- a/sdks/java/extensions/ordered/build.gradle +++ b/sdks/java/extensions/ordered/build.gradle @@ -28,6 +28,12 @@ dependencies { implementation library.java.vendored_guava_32_1_2_jre testImplementation library.java.junit testImplementation library.java.hamcrest + 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/ContiguousSequenceRange.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ContiguousSequenceRange.java new file mode 100644 index 000000000000..c16cf9328dcd --- /dev/null +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ContiguousSequenceRange.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.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.InstantCoder; +import org.apache.beam.sdk.coders.VarLongCoder; +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 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 exclusive end sequence */ + public abstract long getEnd(); + + /** @return latest timestamp of all events in the range */ + public abstract Instant getTimestamp(); + + public static ContiguousSequenceRange of(long start, long end, Instant timestamp) { + return new AutoValue_ContiguousSequenceRange(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( + 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); + } + + @Override + 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 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..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,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 new file mode 100644 index 000000000000..aa12c30a5317 --- /dev/null +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java @@ -0,0 +1,112 @@ +/* + * 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; +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.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.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< + EventKeyT, EventT, ResultT, StateT extends MutableState> + extends PTransform< + PCollection>>>, + PCollectionView> { + + private final Combine.GloballyAsSingletonView< + 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, + int maxElementsBeforeReevaluatingGlobalSequence) { + this.sideInputProducer = sideInputProducer; + this.frequencyOfGeneration = globalSequenceGenerationFrequency; + this.maxElementsBeforeReevaluatingGlobalSequence = maxElementsBeforeReevaluatingGlobalSequence; + } + + @Override + public PCollectionView expand( + PCollection>>> input) { + 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", + // 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()) + .accumulatingFiredPanes() + .withAllowedLateness(input.getWindowingStrategy().getAllowedLateness()) + .triggering( + Repeatedly.forever( + AfterFirst.of( + AfterPane.elementCountAtLeast( + maxElementsBeforeReevaluatingGlobalSequence), + 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 new file mode 100644 index 000000000000..64c2d119c97d --- /dev/null +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java @@ -0,0 +1,276 @@ +/* + * 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; +import org.apache.beam.sdk.coders.Coder; +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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Main Stateful DoFn used to process events in the global sequence mode. + * + * @param + * @param + * @param + * @param + */ +class GlobalSequencesProcessorDoFn< + EventT, EventKeyT, ResultT, StateT extends MutableState> + 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 latestContiguousRangeSideInput; + + private final Duration maxLateness; + + GlobalSequencesProcessorDoFn( + EventExaminer eventExaminer, + Coder eventCoder, + Coder stateCoder, + Coder keyCoder, + TupleTag> mainOutputTupleTag, + TupleTag> statusTupleTag, + Duration statusUpdateFrequency, + TupleTag>>> unprocessedEventTupleTag, + boolean produceStatusUpdateOnEveryEvent, + long maxNumberOfResultsToProduce, + PCollectionView latestContiguousRangeSideInput, + Duration maxLateness) { + super( + eventExaminer, + mainOutputTupleTag, + statusTupleTag, + statusUpdateFrequency, + unprocessedEventTupleTag, + produceStatusUpdateOnEveryEvent, + maxNumberOfResultsToProduce); + + this.latestContiguousRangeSideInput = latestContiguousRangeSideInput; + this.bufferedEventsSpec = StateSpecs.orderedList(eventCoder); + this.processingStateSpec = StateSpecs.value(ProcessingStateCoder.of(keyCoder)); + this.mutableStateSpec = StateSpecs.value(stateCoder); + this.windowClosedSpec = StateSpecs.value(BooleanCoder.of()); + this.maxLateness = maxLateness; + } + + @Override + boolean checkForFirstOrLastEvent() { + return false; + } + + @Override + boolean checkForSequenceGapInBufferedEvents() { + return false; + } + + @ProcessElement + public void processElement( + ProcessContext context, + @Element KV> eventAndSequence, + @StateId(BUFFERED_EVENTS) OrderedListState bufferedEventsProxy, + @AlwaysFetched @StateId(PROCESSING_STATE) + 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); + + EventT event = eventAndSequence.getValue().getValue(); + EventKeyT key = eventAndSequence.getKey(); + long sequence = eventAndSequence.getValue().getKey(); + + if (LOG.isTraceEnabled()) { + LOG.trace(key + ": " + sequence + " lastRange: " + lastContiguousRange); + } + + ProcessingState processingState = processingStateProxy.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(lastContiguousRange); + + if (event == null) { + // 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); + + return; + } + + 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, + mutableStateProxy, + bufferedEventsProxy, + outputReceiver); + + saveStates( + processingStateProxy, + processingState, + mutableStateProxy, + state, + outputReceiver, + window.maxTimestamp()); + + setBatchEmissionTimerIfNeeded(batchEmissionTimer, processingState); + } + + private void setBatchEmissionTimerIfNeeded( + Timer batchEmissionTimer, ProcessingState processingState) { + ContiguousSequenceRange lastCompleteGlobalSequence = processingState.getLastContiguousRange(); + if (lastCompleteGlobalSequence != null + && processingState.thereAreGloballySequencedEventsToBeProcessed()) { + batchEmissionTimer.set(lastCompleteGlobalSequence.getTimestamp().plus(maxLateness)); + } + } + + @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(); + + ContiguousSequenceRange lastContiguousRange = processingState.getLastContiguousRange(); + if (lastContiguousRange == 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; + } + + if (LOG.isTraceEnabled()) { + LOG.trace("Emission timer: " + processingState); + } + + this.numberOfResultsBeforeBundleStart = processingState.getResultCount(); + + state = + processBufferedEventRange( + processingState, + state, + bufferedEventsState, + outputReceiver, + batchEmissionTimer, + lastContiguousRange); + + 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 935647c0e7e5..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 @@ -19,52 +19,44 @@ 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.ProcessingState.ProcessingStateCoder; -import org.apache.beam.sdk.extensions.ordered.UnprocessedEvent.Reason; +import org.apache.beam.sdk.extensions.ordered.OrderedProcessingHandler.OrderedProcessingGlobalSequenceHandler; 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.Flatten; 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.PCollection.IsBounded; +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; -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 * 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. * - * @param - * @param - * @param + *

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 */ @AutoValue @SuppressWarnings({"nullness", "TypeNameShadowing"}) @@ -74,6 +66,18 @@ public abstract class OrderedEventProcessor< 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 type of event key + * @param type of the result object + * @param type of the state to store + * @return the transform + */ public static < EventTypeT, EventKeyTypeT, @@ -129,10 +133,67 @@ public OrderedEventProcessorResult expand( throw new RuntimeException("Unable to get result coder", e); } - PCollectionTuple processingResult = + 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))); + + if (handler instanceof OrderedProcessingGlobalSequenceHandler) { + OrderedProcessingGlobalSequenceHandler + globalSequenceHandler = + (OrderedProcessingGlobalSequenceHandler) handler; + + return expandGlobalSequenceProcessing( + input, + mainOutput, + statusOutput, + unprocessedEventOutput, + handler, + pipeline, + keyCoder, + eventCoder, + stateCoder, + mainOutputCoder, + processingStatusCoder, + unprocessedEventsCoder, + globalSequenceHandler); + } else { + 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 OrderedProcessorDoFn<>( + new SequencePerKeyProcessorDoFn<>( handler.getEventExaminer(), eventCoder, stateCoder, @@ -146,13 +207,6 @@ public OrderedEventProcessorResult expand( .withOutputTags( mainOutput, TupleTagList.of(Arrays.asList(statusOutput, unprocessedEventOutput)))); - - 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), @@ -163,6 +217,84 @@ public OrderedEventProcessorResult expand( 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.getContiguousSequenceRangeReevaluationFrequency(), + globalSequenceHandler + .getMaxElementCountToTriggerContinuousSequenceRangeReevaluation()) + : new GlobalSequenceTracker<>( + globalSequenceHandler.getGlobalSequenceCombiner())); + + if (streamingProcessing) { + PCollection>> tickers = + input.apply( + "Create Tickers", + new PerKeyTickerGenerator<>( + keyCoder, + eventCoder, + globalSequenceHandler.getContiguousSequenceRangeReevaluationFrequency())); + + 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, + input.getWindowingStrategy().getAllowedLateness())) + .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) { SchemaRegistry schemaRegistry = pipeline.getSchemaRegistry(); Coder result; @@ -179,497 +311,16 @@ 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 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 - */ - OrderedProcessorDoFn( - EventExaminer eventExaminer, - Coder eventCoder, - Coder stateCoder, - Coder keyCoder, - TupleTag> mainOutputTupleTag, - TupleTag> statusTupleTag, - Duration statusUpdateFrequency, - TupleTag>>> - unprocessedEventTupleTag, - boolean produceStatusUpdateOnEveryEvent, - long maxNumberOfResultsToProduce) { - this.eventExaminer = eventExaminer; - this.bufferedEventsSpec = StateSpecs.orderedList(eventCoder); - this.mutableStateSpec = StateSpecs.value(stateCoder); - this.processingStateSpec = StateSpecs.value(ProcessingStateCoder.of(keyCoder)); - 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; - } + static class ToTimestampedEventConverter + extends DoFn< + KV>, TimestampedValue>>> { @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) { - - 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; - } - - 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); - // Remove this record also - endClearRange = Instant.ofEpochMilli(eventSequence + 1); - } - - 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); + 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 f61df6254b25..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 @@ -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,10 +31,15 @@ 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 * + *

    In case of global sequence processing, the result also contains PCollectionView of the + * latest contiguous sequence range + * * @param * @param */ @@ -48,6 +55,8 @@ public class OrderedEventProcessorResult implements POutp unprocessedEventPCollection; private final TupleTag>>> unprocessedEventTupleTag; + private final @Nullable PCollectionView latestContiguousRange; + OrderedEventProcessorResult( Pipeline pipeline, PCollection> outputPCollection, @@ -57,6 +66,27 @@ 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 latestContiguousRange) { + this.pipeline = pipeline; this.outputPCollection = outputPCollection; this.outputPCollectionTupleTag = outputPCollectionTupleTag; @@ -64,6 +94,7 @@ public class OrderedEventProcessorResult implements POutp this.eventProcessingStatusTupleTag = eventProcessingStatusTupleTag; this.unprocessedEventPCollection = unprocessedEventPCollection; this.unprocessedEventTupleTag = unprocessedEventTupleTag; + this.latestContiguousRange = latestContiguousRange; } private final Pipeline pipeline; @@ -104,4 +135,8 @@ public PCollection> output() { public PCollection>>> unprocessedEvents() { return unprocessedEventPCollection; } + + 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 444fdb118091..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 @@ -22,7 +22,11 @@ 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.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; @@ -30,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 @@ -217,4 +226,75 @@ public int getMaxOutputElementsPerBundle() { 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 { + + public OrderedProcessingGlobalSequenceHandler( + Class eventTClass, + Class keyTClass, + Class stateTClass, + Class resultTClass) { + super(eventTClass, keyTClass, stateTClass, resultTClass); + } + + /** + * Provide the global sequence combiner. Default is to use {@link DefaultSequenceCombiner}. + * + * @return combiner + */ + public GloballyAsSingletonView< + TimestampedValue>>, ContiguousSequenceRange> + getGlobalSequenceCombiner() { + return Combine.globally(new DefaultSequenceCombiner(getEventExaminer())) + .asSingletonView(); + } + + /** + * 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. + * For Dataflow runner, see {@link this + * Dataflow streaming pipeline option} + * + * @return frequency of reevaluating the {@link ContiguousSequenceRange}. Default - every + * second. + * @see + * OrderedProcessingGlobalSequenceHandler#getMaxElementCountToTriggerContinuousSequenceRangeReevaluation() + */ + public Duration getContiguousSequenceRangeReevaluationFrequency() { + 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/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..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 @@ -30,16 +30,16 @@ 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) @@ -55,8 +55,7 @@ public static OrderedProcessingStatus create( * @return Last sequence processed. If null is returned - no elements for the given key and window * have been processed yet. */ - @Nullable - public abstract Long getLastProcessedSequence(); + public abstract @Nullable Long getLastProcessedSequence(); /** @return Number of events received out of sequence and buffered. */ public abstract long getNumberOfBufferedEvents(); @@ -129,13 +128,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/PerKeyTickerGenerator.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/PerKeyTickerGenerator.java new file mode 100644 index 000000000000..a18ba53f5266 --- /dev/null +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/PerKeyTickerGenerator.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.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; + +/** + * PTransform to generate per key tickers with certain frequency. + * + * @param + * @param + */ +class PerKeyTickerGenerator + extends PTransform< + PCollection>>, + 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 static final String STATE = "state"; + private static final 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, + @AlwaysFetched @StateId(STATE) ValueState state, + @TimerId(TIMER) Timer tickerTimer) { + @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; + } + + // 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/ProcessingState.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessingState.java index 4b591a37faab..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 @@ -51,6 +51,8 @@ class ProcessingState { private long resultCount; + @Nullable private ContiguousSequenceRange lastCompleteGlobalSequence; + private KeyT key; public ProcessingState(KeyT key) { @@ -59,6 +61,7 @@ public ProcessingState(KeyT key) { this.lastOutputSequence = null; this.earliestBufferedSequence = null; this.latestBufferedSequence = null; + this.lastCompleteGlobalSequence = null; } /** @@ -130,6 +133,15 @@ public KeyT getKey() { return key; } + public @Nullable ContiguousSequenceRange getLastContiguousRange() { + return lastCompleteGlobalSequence; + } + + public void setLastCompleteGlobalSequence( + @Nullable ContiguousSequenceRange lastCompleteGlobalSequence) { + this.lastCompleteGlobalSequence = lastCompleteGlobalSequence; + } + /** * Current event matched the sequence and was processed. * @@ -229,6 +241,32 @@ 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; } @@ -274,6 +312,23 @@ public long resultsProducedInBundle(long numberOfResultsBeforeBundleStart) { return resultCount - numberOfResultsBeforeBundleStart; } + 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 + // without a chance to run and produce output. + return; + } + this.lastCompleteGlobalSequence = updated; + } + + public boolean thereAreGloballySequencedEventsToBeProcessed() { + return bufferedEventCount > 0 + && lastCompleteGlobalSequence != null + && earliestBufferedSequence != null + && earliestBufferedSequence < lastCompleteGlobalSequence.getEnd(); + } + /** * Coder for the processing status. * @@ -287,6 +342,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(ContiguousSequenceRange.CompletedSequenceRangeCoder.of()); + private Coder keyCoder; private ProcessingStateCoder(Coder keyCoder) { @@ -308,6 +366,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.getLastContiguousRange(), outStream); } @Override @@ -321,17 +380,23 @@ public ProcessingState decode(InputStream inStream) throws IOException { long resultCount = LONG_CODER.decode(inStream); boolean isLastEventReceived = BOOLEAN_CODER.decode(inStream); KeyT key = keyCoder.decode(inStream); - - return 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; } @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 new file mode 100644 index 000000000000..a05b0829074a --- /dev/null +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java @@ -0,0 +1,427 @@ +/* + * 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; +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.joda.time.Duration; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Base DoFn for processing ordered events. + * + * @param type of the events to process + * @param event key type + * @param state type + */ +abstract class ProcessorDoFn< + EventT, EventKeyT, ResultT, StateT 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 = 0L; + + 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; + } + + /** @return true if each event needs to be examined. */ + abstract boolean checkForFirstOrLastEvent(); + + /** + * Process the just received event. + * + * @return newly created or updated State. If null is returned - the event wasn't processed. + */ + protected @javax.annotation.Nullable StateT processNewEvent( + long currentSequence, + 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. + // 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; + } + + StateT state; + 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? + state = eventExaminer.createStateOnInitialEvent(currentEvent); + + processingState.eventAccepted(currentSequence, thisIsTheLastEvent); + + ResultT 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; + } + + ResultT 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. We don't need to save it. + return null; + } + + protected void saveStates( + ValueState> processingStatusState, + ProcessingState processingStatus, + ValueState currentStateState, + @Nullable StateT 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); + } + } + + 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. + 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, + Instant statusTimestamp) { + if (LOG.isTraceEnabled()) { + LOG.trace("Emitting status for: " + processingState.getKey() + ", " + processingState); + } + 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) + >= maxNumberOfResultsToProduce; + if (exceeded) { + 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 + largeBatchEmissionTimer.offset(Duration.millis(1)).setRelative(); + } + return exceeded; + } + + 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); + } + + abstract boolean checkForSequenceGapInBufferedEvents(); + + @Nullable + StateT processBufferedEventRange( + ProcessingState processingState, + @Nullable StateT state, + OrderedListState bufferedEventsState, + MultiOutputReceiver outputReceiver, + Timer largeBatchEmissionTimer, + ContiguousSequenceRange contiguousSequenceRange) { + 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); + + 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(); + + 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)) { + // 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( + KV.of( + processingState.getKey(), + KV.of( + 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. + continue; + } + + Long lastOutputSequence = processingState.getLastOutputSequence(); + boolean currentEventIsNextInSequence = + lastOutputSequence != null && eventSequence == lastOutputSequence + 1; + boolean continueProcessing = + checkForSequenceGapInBufferedEvents() + ? currentEventIsNextInSequence + : (eventSequence < contiguousSequenceRange.getEnd() || currentEventIsNextInSequence); + if (!continueProcessing) { + 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) { + if (LOG.isTraceEnabled()) { + LOG.trace("Creating a new state: " + processingState.getKey() + " " + bufferedEvent); + } + state = eventExaminer.createStateOnInitialEvent(bufferedEvent); + } else { + if (LOG.isTraceEnabled()) { + LOG.trace("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; + } + + ResultT 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 new file mode 100644 index 000000000000..878a0664ac87 --- /dev/null +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java @@ -0,0 +1,294 @@ +/* + * 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; +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.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.TupleTag; +import org.joda.time.Duration; +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< + 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; + + @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()); + } + + @Override + boolean checkForFirstOrLastEvent() { + return true; + } + + @Override + boolean checkForSequenceGapInBufferedEvents() { + return true; + } + + @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.isTraceEnabled()) { + LOG.trace("Processing for key '" + processingState.getKey() + "' is completed."); + } + return result; + } + + /** 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; + } + + processBufferedEventRange( + processingState, + state, + bufferedEventsState, + outputReceiver, + largeBatchEmissionTimer, + ContiguousSequenceRange.EMPTY); + } + + @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) { + + processStatusTimerEvent( + outputReceiver, statusEmissionTimer, windowClosedState, processingStateState); + } + + @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/UnprocessedEvent.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/UnprocessedEvent.java index 2131ef384e22..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 @@ -72,7 +72,8 @@ 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/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..32e5cbc36e4e --- /dev/null +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.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.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; +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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * 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)}. + * + *

    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< + 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; + }; + 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 ContiguousSequenceRange extractOutput(SequenceRangeAccumulator accum) { + ContiguousSequenceRange result = accum.largestContinuousRange(); + if (LOG.isTraceEnabled()) { + LOG.trace("Returning completed sequence range: " + result); + } + return result; + } + + @Override + public @UnknownKeyFor @NonNull @Initialized Coder getAccumulatorCoder( + @UnknownKeyFor @NonNull @Initialized CoderRegistry registry, + @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 new file mode 100644 index 000000000000..89dc912afc90 --- /dev/null +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulator.java @@ -0,0 +1,296 @@ +/* + * 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; +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.coders.NullableCoder; +import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.coders.VarLongCoder; +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; + +/** Default accumulator used to combine sequence ranges. */ +public class SequenceRangeAccumulator { + + 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; + + 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); + } + + 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); + } else if (initialSequence != null && sequence <= initialSequence) { + // No need to add anything lower than the initial sequence to the accumulator. + return; + } + + long lowerBound = sequence, upperBound = sequence; + + Entry> lowerRange = data.floorEntry(sequence); + if (lowerRange != null) { + long inclusiveUpperBoundary = lowerRange.getValue().getLeft(); + if (sequence <= inclusiveUpperBoundary) { + // Duplicate. No need to adjust the timestamp. + return; + } + + if (inclusiveUpperBoundary + 1 == sequence) { + // The new element extends the lower range. Remove the range. + timestamp = max(timestamp, lowerRange.getValue().getValue()); + lowerBound = lowerRange.getKey(); + data.remove(lowerRange.getKey()); + } + } + + long nextSequenceNumber = sequence + 1; + 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(); + data.remove(nextSequenceNumber); + } + + data.put(lowerBound, Pair.of(upperBound, timestamp)); + } + + private void clearRangesBelowInitialSequence(long sequence, Instant timestamp) { + // First, adjust the current range, if any + 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. + data.remove(lowerRange.getKey()); + data.put( + sequence, + Pair.of( + lowerRange.getValue().getKey(), max(timestamp, lowerRange.getValue().getValue()))); + } + data.subMap(Long.MIN_VALUE, sequence).clear(); + } + + public ContiguousSequenceRange largestContinuousRange() { + if (initialSequence == null) { + return ContiguousSequenceRange.EMPTY; + } + + Entry> firstEntry = data.firstEntry(); + if (firstEntry == null) { + throw new IllegalStateException("First entry is null when initial sequence is set."); + } + Long start = firstEntry.getKey(); + Long end = firstEntry.getValue().getLeft(); + Instant latestTimestamp = firstEntry.getValue().getRight(); + // 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() { + return data.size(); + } + + 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.data.firstEntry(); + if (firstEntry != null) { + Instant timestampOfTheInitialRange = firstEntry.getValue().getRight(); + clearRangesBelowInitialSequence(newInitialSequence, timestampOfTheInitialRange); + } + } + + 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); + for (Pair value : rangesToRemove.values()) { + result = result.isAfter(value.getRight()) ? result : value.getRight(); + } + rangesToRemove.clear(); + 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 { + + 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(); + + @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 { + 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/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; 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..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 @@ -16,7 +16,9 @@ * 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/OrderedEventProcessorGlobalSequenceTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java new file mode 100644 index 000000000000..98bc7591f4d7 --- /dev/null +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java @@ -0,0 +1,534 @@ +/* + * 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; +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; +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 OrderedEventProcessorTestBase { + + 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 = { + 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<>(); + 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, + expectedOutput, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT, + ContiguousSequenceRange.of(0, 6, new Instant())); + } + + @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") + }; + + 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, + expectedOutput, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT, + ContiguousSequenceRange.of(0, 9, new Instant())); + } + + @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"), + }; + + 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, + expectedOutput, + duplicates, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT, + ContiguousSequenceRange.of(0, 4, new Instant())); + } + + @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.before_initial_sequence)))); + duplicates.add( + KV.of( + "id-1", + KV.of(-2L, UnprocessedEvent.create("early-2", Reason.before_initial_sequence)))); + + testGlobalSequenceProcessing( + events, + expectedOutput, + duplicates, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT, + ContiguousSequenceRange.of(0, 4, new Instant())); + } + + @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"), + }; + + // 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( + KV.of( + "id-1", + KV.of( + 2L, + UnprocessedEvent.create(StringBuilderState.BAD_VALUE, Reason.exception_thrown)))); + + testGlobalSequenceProcessing( + events, + expectedOutput, + failedEvents, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + INITIAL_SEQUENCE_OF_0, + 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 + ContiguousSequenceRange.of(0, 4, new Instant())); + } + + @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"), + }; + + 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, + expectedOutput, + EMISSION_FREQUENCY_ON_EVERY_OTHER_EVENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT, + ContiguousSequenceRange.of(0, 6, new Instant())); + } + + @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); + + 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())); + } + + testGlobalSequenceProcessing( + events.toArray(new Event[events.size()]), + expectedOutput, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + 1L /* This dataset assumes 1 as the starting sequence */, + maxResultsPerOutput, + ContiguousSequenceRange.of(1, sequences.length + 1, new Instant())); + } + + @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); + + String key = "id-1"; + + for (long sequence : sequences) { + 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, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + 1L /* This dataset assumes 1 as the starting sequence */, + maxResultsPerOutput, + ContiguousSequenceRange.of(1, 11, new Instant())); + } + + @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") + }; + + 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<>(); + unprocessedEvents.add( + KV.of( + "id-1", + KV.of( + Long.MAX_VALUE, + UnprocessedEvent.create("d", Reason.sequence_id_outside_valid_range)))); + + testGlobalSequenceProcessing( + events, + expectedOutput, + unprocessedEvents, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT, + ContiguousSequenceRange.of(0, 3, Instant.now())); + } + + @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"), + Event.create(2, "id-1", StringEventExaminer.LAST_INPUT) + }; + + 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, + expectedOutput, + EMISSION_FREQUENCY_ON_EVERY_ELEMENT, + INITIAL_SEQUENCE_OF_0, + LARGE_MAX_RESULTS_PER_OUTPUT, + ContiguousSequenceRange.of(0, 3, new Instant())); + } + + private void testGlobalSequenceProcessing( + Event[] events, + Collection> expectedOutput, + int emissionFrequency, + long initialSequence, + int maxResultsPerOutput, + ContiguousSequenceRange expectedLastCompleteRange) + throws CannotProvideCoderException { + testGlobalSequenceProcessing( + events, + expectedOutput, + NO_EXPECTED_DLQ_EVENTS, + emissionFrequency, + initialSequence, + maxResultsPerOutput, + expectedLastCompleteRange); + } + + private void testGlobalSequenceProcessing( + Event[] events, + Collection> expectedOutput, + Collection>>> expectedUnprocessedEvents, + int emissionFrequency, + long initialSequence, + int maxResultsPerOutput, + ContiguousSequenceRange expectedLastCompleteRange) + throws CannotProvideCoderException { + // Test a streaming pipeline + doTest( + events, + null /* expectedStatuses */, + expectedOutput, + expectedUnprocessedEvents, + emissionFrequency, + initialSequence, + maxResultsPerOutput, + false /* produceStatusOnEveryEvent */, + STREAMING, + GLOBAL_SEQUENCE, + expectedLastCompleteRange); + + // Test a batch pipeline + 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."); + } + } + + @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)))) + .advanceProcessingTime(Duration.standardMinutes(15)) + .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)))); + + 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); + + 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")); + + // 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); + + 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/OrderedEventProcessorPerKeySequenceTest.java similarity index 71% 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/OrderedEventProcessorPerKeySequenceTest.java index 6a24021ad667..6909a3bb992c 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/OrderedEventProcessorPerKeySequenceTest.java @@ -20,82 +20,24 @@ 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; import org.apache.beam.sdk.extensions.ordered.UnprocessedEvent.Reason; import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.SerializableMatcher; -import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestStream; -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.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; -import org.hamcrest.BaseMatcher; -import org.hamcrest.Description; 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; -/** - * 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 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; - public static final boolean DONT_PRODUCE_STATUS_ON_EVERY_EVENT = false; - public static final int LARGE_MAX_RESULTS_PER_OUTPUT = 1000; - public static final int EMISSION_FREQUENCY_ON_EVERY_OTHER_EVENT = 2; - public static final boolean PRODUCE_STATUS_ON_EVERY_EVENT = true; - public static final boolean STREAMING = true; - 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(); - - static class MapEventsToKV extends DoFn>> { - - @ProcessElement - public void convert( - @Element Event event, OutputReceiver>> outputReceiver) { - outputReceiver.output(KV.of(event.getKey(), KV.of(event.getSequence(), event.getValue()))); - } - } - - static class MapStringBufferStateToString - extends DoFn, KV> { - - @ProcessElement - public void map( - @Element KV element, - OutputReceiver> outputReceiver) { - outputReceiver.output(KV.of(element.getKey(), element.getValue().toString())); - } - } +public class OrderedEventProcessorPerKeySequenceTest extends OrderedEventProcessorTestBase { @Test public void testPerfectOrderingProcessing() throws CannotProvideCoderException { @@ -142,7 +84,7 @@ public void testPerfectOrderingProcessing() throws CannotProvideCoderException { expectedOutput.add(KV.of("id-2", "a")); expectedOutput.add(KV.of("id-2", "ab")); - testProcessing( + testPerKeySequenceProcessing( events, expectedStatuses, expectedOutput, @@ -203,7 +145,7 @@ public void testOutOfSequenceProcessing() throws CannotProvideCoderException { expectedOutput.add(KV.of("id-2", "abcd")); expectedOutput.add(KV.of("id-2", "abcde")); - testProcessing( + testPerKeySequenceProcessing( events, expectedStatuses, expectedOutput, @@ -235,7 +177,7 @@ public void testUnfinishedProcessing() throws CannotProvideCoderException { expectedOutput.add(KV.of("id-2", "a")); expectedOutput.add(KV.of("id-2", "ab")); - testProcessing(events, expectedStatuses, expectedOutput, 1, 0, 1000, false); + testPerKeySequenceProcessing(events, expectedStatuses, expectedOutput, 1, 0, 1000, false); } @Test @@ -275,7 +217,7 @@ public void testHandlingOfDuplicateSequences() throws CannotProvideCoderExceptio 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( + testPerKeySequenceProcessing( events, expectedStatuses, expectedOutput, @@ -311,7 +253,7 @@ public void testHandlingOfCheckedExceptions() throws CannotProvideCoderException 2L, UnprocessedEvent.create(StringBuilderState.BAD_VALUE, Reason.exception_thrown)))); - testProcessing( + testPerKeySequenceProcessing( events, expectedStatuses, expectedOutput, @@ -346,7 +288,7 @@ public void testProcessingWithEveryOtherResultEmission() throws CannotProvideCod // Skipped KV.of("id-1", "abcd"), expectedOutput.add(KV.of("id-2", "a")); // Skipped KV.of("id-2", "ab") - testProcessing( + testPerKeySequenceProcessing( events, expectedStatuses, expectedOutput, @@ -428,7 +370,7 @@ public void testLargeBufferedOutputInTimer() throws CannotProvideCoderException 0, false))); - testProcessing( + testPerKeySequenceProcessing( events.toArray(new Event[events.size()]), expectedStatuses, expectedOutput, @@ -523,7 +465,7 @@ public void testSequenceGapProcessingInBufferedOutput() throws CannotProvideCode OrderedProcessingStatus.create( 10L, 0, null, null, numberOfReceivedEvents, 10L, 0, false))); - testProcessing( + testPerKeySequenceProcessing( events.toArray(new Event[events.size()]), expectedStatuses, expectedOutput, @@ -558,7 +500,7 @@ public void testHandlingOfMaxSequenceNumber() throws CannotProvideCoderException Long.MAX_VALUE, UnprocessedEvent.create("c", Reason.sequence_id_outside_valid_range)))); - testProcessing( + testPerKeySequenceProcessing( events, expectedStatuses, expectedOutput, @@ -589,7 +531,7 @@ public void testProcessingOfTheLastInput() throws CannotProvideCoderException { expectedOutput.add(KV.of("id-1", "ab")); expectedOutput.add(KV.of("id-1", "ab" + StringEventExaminer.LAST_INPUT)); - testProcessing( + testPerKeySequenceProcessing( events, expectedStatuses, expectedOutput, @@ -599,6 +541,65 @@ public void testProcessingOfTheLastInput() throws CannotProvideCoderException { 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, + false, + ContiguousSequenceRange.EMPTY); + + // Test a batch pipeline + doTest( + events, + expectedStatuses, + expectedOutput, + expectedUnprocessedEvents, + emissionFrequency, + initialSequence, + maxResultsPerOutput, + produceStatusOnEveryEvent, + BATCH, + false, + ContiguousSequenceRange.EMPTY); + } + @Test public void testWindowedProcessing() throws CannotProvideCoderException { @@ -684,223 +685,4 @@ public void testWindowedProcessing() throws CannotProvideCoderException { 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. - * - * @param events - * @param expectedStatuses - * @param expectedOutput - * @param expectedUnprocessedEvents - * @param emissionFrequency - * @param initialSequence - * @param maxResultsPerOutput - * @param produceStatusOnEveryEvent - * @param streaming - * @throws @UnknownKeyFor @NonNull @Initialized CannotProvideCoderException - */ - private void doTest( - Event[] events, - Collection> expectedStatuses, - Collection> expectedOutput, - Collection>>> expectedUnprocessedEvents, - int emissionFrequency, - long initialSequence, - int maxResultsPerOutput, - boolean produceStatusOnEveryEvent, - boolean streaming) - throws @UnknownKeyFor @NonNull @Initialized CannotProvideCoderException { - - Pipeline pipeline = streaming ? streamingPipeline : batchPipeline; - - PCollection rawInput = - streaming - ? createStreamingPCollection(pipeline, events) - : createBatchPCollection(pipeline, events); - PCollection>> input = - rawInput.apply("To KV", ParDo.of(new MapEventsToKV())); - - StringBufferOrderedProcessingHandler handler = - new StringBufferOrderedProcessingHandler(emissionFrequency, initialSequence); - handler.setMaxOutputElementsPerBundle(maxResultsPerOutput); - if (produceStatusOnEveryEvent) { - handler.setProduceStatusUpdateOnEveryEvent(true); - // This disables status updates emitted on timers. - handler.setStatusUpdateFrequency(null); - } else { - handler.setStatusUpdateFrequency( - streaming ? Duration.standardMinutes(5) : Duration.standardSeconds(1)); - } - OrderedEventProcessor orderedEventProcessor = - OrderedEventProcessor.create(handler); - - OrderedEventProcessorResult processingResult = - input.apply("Process Events", orderedEventProcessor); - - 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 - // 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()) - .containsInAnyOrder(expectedStatuses); - } - - // This is a temporary workaround until PAssert changes. - boolean unprocessedEventsHaveExceptionStackTrace = false; - for (KV>> event : expectedUnprocessedEvents) { - if (event.getValue().getValue().getReason() == Reason.exception_thrown) { - unprocessedEventsHaveExceptionStackTrace = true; - break; - } - } - - if (unprocessedEventsHaveExceptionStackTrace) { - PAssert.thatSingleton( - "Unprocessed event count", - processingResult - .unprocessedEvents() - .apply( - "Window", - Window.>>>into( - new GlobalWindows()) - .triggering(Repeatedly.forever(AfterWatermark.pastEndOfWindow())) - .discardingFiredPanes()) - .apply("Count", Count.globally())) - .isEqualTo((long) expectedUnprocessedEvents.size()); - } else { - PAssert.that("Unprocessed events match", processingResult.unprocessedEvents()) - .containsInAnyOrder(expectedUnprocessedEvents); - } - pipeline.run(); - } - - private @UnknownKeyFor @NonNull @Initialized PCollection createBatchPCollection( - Pipeline pipeline, Event[] events) { - return pipeline - .apply("Create Batch Events", Create.of(Arrays.asList(events))) - .apply("Reshuffle", Reshuffle.viaRandomKey()); - } - - private @UnknownKeyFor @NonNull @Initialized PCollection createStreamingPCollection( - Pipeline pipeline, Event[] events) - throws @UnknownKeyFor @NonNull @Initialized CannotProvideCoderException { - Instant now = Instant.now().minus(Duration.standardMinutes(20)); - TestStream.Builder messageFlow = - TestStream.create(pipeline.getCoderRegistry().getCoder(Event.class)) - .advanceWatermarkTo(now); - - int delayInMilliseconds = 0; - for (Event e : events) { - messageFlow = - messageFlow - .advanceWatermarkTo(now.plus(Duration.millis(++delayInMilliseconds))) - .addElements(e); - } - - // Needed to force the processing time based timers. - messageFlow = messageFlow.advanceProcessingTime(Duration.standardMinutes(15)); - return pipeline.apply("Create Streaming Events", messageFlow.advanceWatermarkToInfinity()); - } - - /** - * Unprocessed event's explanation contains stacktraces which makes tests very brittle because it - * requires hardcoding the line numbers in the code. We use this matcher to only compare on the - * first line of the explanation. - */ - static class UnprocessedEventMatcher - extends BaseMatcher>>> - implements SerializableMatcher>>> { - - private KV>> element; - - public UnprocessedEventMatcher(KV>> element) { - this.element = element; - } - - @Override - public boolean matches(Object actual) { - KV>> toMatch = - (KV>>) actual; - - UnprocessedEvent originalEvent = element.getValue().getValue(); - UnprocessedEvent eventToMatch = toMatch.getValue().getValue(); - - return element.getKey().equals(toMatch.getKey()) - && element.getValue().getKey().equals(toMatch.getValue().getKey()) - && originalEvent.getEvent().equals(eventToMatch.getEvent()) - && originalEvent.getReason() == eventToMatch.getReason() - && normalizeExplanation(originalEvent.getExplanation()) - .equals(normalizeExplanation(eventToMatch.getExplanation())); - } - - @Override - public void describeTo(Description description) { - description.appendText("Just some text..."); - } - - static String normalizeExplanation(String value) { - if (value == null) { - return ""; - } - String firstLine = value.split("\n", 1)[0]; - if (firstLine.contains("Exception")) { - return firstLine; - } - return value; - } - } } 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 new file mode 100644 index 000000000000..fd651b919df1 --- /dev/null +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorTestBase.java @@ -0,0 +1,395 @@ +/* + * 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 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.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; +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; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.TestStream; +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.PCollection.IsBounded; +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; +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Rule; + +/** + * 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. + */ +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; + public static final boolean DONT_PRODUCE_STATUS_ON_EVERY_EVENT = false; + public static final int LARGE_MAX_RESULTS_PER_OUTPUT = 1000; + public static final int EMISSION_FREQUENCY_ON_EVERY_OTHER_EVENT = 2; + public static final boolean PRODUCE_STATUS_ON_EVERY_EVENT = true; + public static final boolean STREAMING = true; + 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(); + + 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 + public void convert( + @Element Event event, OutputReceiver>> outputReceiver) { + outputReceiver.output(KV.of(event.getKey(), KV.of(event.getSequence(), event.getValue()))); + } + } + + static class MapStringBufferStateToString + extends DoFn, KV> { + + @ProcessElement + public void map( + @Element KV element, + OutputReceiver> outputReceiver) { + outputReceiver.output(KV.of(element.getKey(), element.getValue().toString())); + } + } + + /** + * The majority of the tests use this method. Testing is done in the global window. + * + * @throws @UnknownKeyFor @NonNull @Initialized CannotProvideCoderException + */ + protected void doTest( + Event[] events, + @Nullable Collection> expectedStatuses, + Collection> expectedOutput, + Collection>>> expectedUnprocessedEvents, + int emissionFrequency, + long initialSequence, + int maxResultsPerOutput, + boolean produceStatusOnEveryEvent, + boolean streaming, + boolean isGlobalSequence, + @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) + : createBatchPCollection(pipeline, events); + PCollection>> input = + rawInput.apply("To KV", ParDo.of(new MapEventsToKV())); + + OrderedProcessingHandler handler = + isGlobalSequence + ? new StringBufferOrderedProcessingWithGlobalSequenceHandler( + emissionFrequency, initialSequence) + : new StringBufferOrderedProcessingHandler(emissionFrequency, initialSequence); + handler.setMaxOutputElementsPerBundle(maxResultsPerOutput); + if (produceStatusOnEveryEvent) { + handler.setProduceStatusUpdateOnEveryEvent(true); + // This disables status updates emitted on timers. + handler.setStatusUpdateFrequency(null); + } else { + handler.setStatusUpdateFrequency( + streaming ? Duration.standardMinutes(5) : Duration.standardSeconds(1)); + } + + OrderedEventProcessor orderedEventProcessor = + OrderedEventProcessor.create(handler); + + OrderedEventProcessorResult processingResult = + input.apply("Process Events", orderedEventProcessor); + + 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 + // 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()) + .containsInAnyOrder(expectedStatuses); + } + + // This is a temporary workaround until PAssert changes. + boolean unprocessedEventsHaveExceptionStackTrace = false; + for (KV>> event : expectedUnprocessedEvents) { + if (event.getValue().getValue().getReason() == Reason.exception_thrown) { + unprocessedEventsHaveExceptionStackTrace = true; + break; + } + } + + if (unprocessedEventsHaveExceptionStackTrace) { + PAssert.thatSingleton( + "Unprocessed event count", + processingResult + .unprocessedEvents() + .apply( + "Window", + Window.>>>into( + new GlobalWindows()) + .triggering(Repeatedly.forever(AfterWatermark.pastEndOfWindow())) + .discardingFiredPanes()) + .apply("Count", Count.globally())) + .isEqualTo((long) expectedUnprocessedEvents.size()); + } else { + PAssert.that("Unprocessed events match", processingResult.unprocessedEvents()) + .containsInAnyOrder(expectedUnprocessedEvents); + } + + 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)); + } + pipeline.run(); + } + + static class LastExpectedGlobalSequenceRangeMatcher + implements SerializableFunction, Void> { + + private final long expectedStart; + private final long expectedEnd; + + LastExpectedGlobalSequenceRangeMatcher(ContiguousSequenceRange expected) { + this.expectedStart = expected.getStart(); + this.expectedEnd = expected.getEnd(); + } + + @Override + public Void apply(Iterable input) { + StringBuilder listOfRanges = new StringBuilder("["); + Iterator iterator = input.iterator(); + ContiguousSequenceRange 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 + .apply("Create Batch Events", Create.of(Arrays.asList(events))) + .apply("Reshuffle", Reshuffle.viaRandomKey()); + } + + private @UnknownKeyFor @NonNull @Initialized PCollection createStreamingPCollection( + Pipeline pipeline, Event[] events) + throws @UnknownKeyFor @NonNull @Initialized CannotProvideCoderException { + Instant now = Instant.now().minus(Duration.standardMinutes(20)); + TestStream.Builder messageFlow = + TestStream.create(pipeline.getCoderRegistry().getCoder(Event.class)) + .advanceWatermarkTo(now); + + int delayInMilliseconds = 0; + for (Event e : events) { + messageFlow = + messageFlow + .advanceWatermarkTo(now.plus(Duration.millis(++delayInMilliseconds))) + .addElements(e); + } + + // Needed to force the processing time based timers. + messageFlow = messageFlow.advanceProcessingTime(Duration.standardMinutes(15)); + return pipeline.apply("Create Streaming Events", messageFlow.advanceWatermarkToInfinity()); + } + + /** + * Unprocessed event's explanation contains stacktraces which makes tests very brittle because it + * requires hardcoding the line numbers in the code. We use this matcher to only compare on the + * first line of the explanation. + */ + static class UnprocessedEventMatcher + extends BaseMatcher>>> + implements SerializableMatcher>>> { + + private KV>> element; + + public UnprocessedEventMatcher(KV>> element) { + this.element = element; + } + + @Override + public boolean matches(Object actual) { + KV>> toMatch = + (KV>>) actual; + + UnprocessedEvent originalEvent = element.getValue().getValue(); + UnprocessedEvent eventToMatch = toMatch.getValue().getValue(); + + return element.getKey().equals(toMatch.getKey()) + && element.getValue().getKey().equals(toMatch.getValue().getKey()) + && originalEvent.getEvent().equals(eventToMatch.getEvent()) + && originalEvent.getReason() == eventToMatch.getReason() + && normalizeExplanation(originalEvent.getExplanation()) + .equals(normalizeExplanation(eventToMatch.getExplanation())); + } + + @Override + public void describeTo(Description description) { + description.appendText("Just some text..."); + } + + static String normalizeExplanation(String value) { + if (value == null) { + return ""; + } + String firstLine = value.split("\n", 1)[0]; + if (firstLine.contains("Exception")) { + return firstLine; + } + 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) { + 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)); + } 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>, ContiguousSequenceRange> { + + @ProcessElement + public void produceCompletedRange( + @SideInput("lastCompletedSequence") ContiguousSequenceRange 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..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 @@ -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/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..0e5b0b7c819a --- /dev/null +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorCoderTest.java @@ -0,0 +1,71 @@ +/* + * 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; + +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 = SequenceRangeAccumulatorCoder.of(); + + @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); + } +} 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..4082ce6de758 --- /dev/null +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/combiner/SequenceRangeAccumulatorTest.java @@ -0,0 +1,400 @@ +/* + * 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; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.beam.sdk.extensions.ordered.ContiguousSequenceRange; +import org.joda.time.Instant; +import org.junit.Assert; +import org.junit.Test; + +public class SequenceRangeAccumulatorTest { + + // Atomic just in case tests are run in parallel + private static final 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; + 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() { + Event[] events = + new Event[] { + new Event(1, nextTimestamp(), true), + new Event(2, nextTimestamp()), + new Event(3, nextTimestamp()) + }; + + doTestAccumulation(events, ContiguousSequenceRange.of(1, 4, 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) + }; + + Instant timestampOfEventNumber1 = eventTimestamp(events, 1); + doTestAccumulation(events, ContiguousSequenceRange.of(1, 4, 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()), + }; + + doTestAccumulation(events, ContiguousSequenceRange.of(1, 4, 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()), + }; + + doTestAccumulation(events, ContiguousSequenceRange.of(1, 4, 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()), + }; + + doTestAccumulation(events, ContiguousSequenceRange.EMPTY, 2); + } + + @Test + public void testNoEventsAccumulation() { + 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()), + + // Second range + new Event(5, nextTimestamp()), + new Event(6, nextTimestamp()), + + // This event should prune everything below + new Event(7, nextTimestamp(), true), + }; + + doTestAccumulation(events, ContiguousSequenceRange.of(7, 8, 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, ContiguousSequenceRange.of(7, 9, timestampOfTheLastEvent), 1); + } + + 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("Number of ranges", expectedNumberOfRanges, accumulator.numberOfRanges()); + } + + @Test + public void testEmptyMerge() { + Event[] set1 = new Event[] {}; + Event[] set2 = new Event[] {}; + + ContiguousSequenceRange expectedResult = ContiguousSequenceRange.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[] {}; + + ContiguousSequenceRange expectedResult = + ContiguousSequenceRange.of(1, 4, 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()) + }; + + ContiguousSequenceRange expectedResult = + ContiguousSequenceRange.of(1, 3, 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()) + }; + + ContiguousSequenceRange expectedResult = ContiguousSequenceRange.EMPTY; + int expectedNumberOfRanges = 2; + + doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); + } + + @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, 7, 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()) + }; + + ContiguousSequenceRange expectedResult = + ContiguousSequenceRange.of(3, 7, eventTimestamp(set2, 6L)); + int expectedNumberOfRanges = 1; + + doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); + } + + @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, 7, 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, 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()); + } + } + + @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, 7, eventTimestamp(set1, 2L)); + int expectedNumberOfRanges = 1; + doTestMerging(set1, set2, expectedResult, expectedNumberOfRanges); + } + + 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 + mergeAndTest(set1, set2, expectedResult, expectedNumberOfRanges, "set1"); + mergeAndTest(set2, set1, expectedResult, expectedNumberOfRanges, "set2"); + } + + private static void mergeAndTest( + Event[] set1, + Event[] set2, + ContiguousSequenceRange 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()); + } +}