damccorm commented on code in PR #32540: URL: https://github.com/apache/beam/pull/32540#discussion_r1791932081
########## sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java: ########## @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.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 <EventKeyT> type of event key + * @param <EventT> type of event + * @param <ResultT> type of processing result + * @param <StateT> type of state + */ +class GlobalSequenceTracker< + EventKeyT, EventT, ResultT, StateT extends MutableState<EventT, ResultT>> + extends PTransform< + PCollection<TimestampedValue<KV<EventKeyT, KV<Long, EventT>>>>, + PCollectionView<ContiguousSequenceRange>> { + + private final Combine.GloballyAsSingletonView< + TimestampedValue<KV<EventKeyT, KV<Long, EventT>>>, ContiguousSequenceRange> + sideInputProducer; + private final @Nullable Duration frequencyOfGeneration; + + public GlobalSequenceTracker( + Combine.GloballyAsSingletonView< + TimestampedValue<KV<EventKeyT, KV<Long, EventT>>>, ContiguousSequenceRange> + sideInputProducer) { + this.sideInputProducer = sideInputProducer; + this.frequencyOfGeneration = null; + } + + public GlobalSequenceTracker( + Combine.GloballyAsSingletonView< + TimestampedValue<KV<EventKeyT, KV<Long, EventT>>>, ContiguousSequenceRange> + sideInputProducer, + Duration globalSequenceGenerationFrequency) { + this.sideInputProducer = sideInputProducer; + this.frequencyOfGeneration = globalSequenceGenerationFrequency; + } + + @Override + public PCollectionView<ContiguousSequenceRange> expand( + PCollection<TimestampedValue<KV<EventKeyT, KV<Long, EventT>>>> 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.<TimestampedValue<KV<EventKeyT, KV<Long, EventT>>>>into( + (WindowFn<? super TimestampedValue<KV<EventKeyT, KV<Long, EventT>>>, ?>) + input.getWindowingStrategy().getWindowFn()) + .accumulatingFiredPanes() + // TODO: verify that we don't need to have the lateness parameterized + .withAllowedLateness(Duration.ZERO) Review Comment: I think this should be pulled from the input windowing strategy, right? We can pull it from https://github.com/slilichenko/beam/blob/3035e5a8fa05ca2c22eb7097226a9312f3bd6481/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowingStrategy.java#L137 Otherwise, allowed late data might be dropped ########## sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/UnprocessedEvent.java: ########## @@ -68,6 +68,7 @@ static <EventT> UnprocessedEvent<EventT> create( return new AutoValue_UnprocessedEvent<>(event, reason, failureDetails); } + // TODO: do we need another reason, "before the initial event"? Review Comment: Probably this would be good to add ########## sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java: ########## @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.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 <EventKeyT> type of event key + * @param <EventT> type of event + * @param <ResultT> type of processing result + * @param <StateT> type of state + */ +class GlobalSequenceTracker< + EventKeyT, EventT, ResultT, StateT extends MutableState<EventT, ResultT>> + extends PTransform< + PCollection<TimestampedValue<KV<EventKeyT, KV<Long, EventT>>>>, + PCollectionView<ContiguousSequenceRange>> { + + private final Combine.GloballyAsSingletonView< + TimestampedValue<KV<EventKeyT, KV<Long, EventT>>>, ContiguousSequenceRange> + sideInputProducer; + private final @Nullable Duration frequencyOfGeneration; + + public GlobalSequenceTracker( + Combine.GloballyAsSingletonView< + TimestampedValue<KV<EventKeyT, KV<Long, EventT>>>, ContiguousSequenceRange> + sideInputProducer) { + this.sideInputProducer = sideInputProducer; + this.frequencyOfGeneration = null; + } + + public GlobalSequenceTracker( + Combine.GloballyAsSingletonView< + TimestampedValue<KV<EventKeyT, KV<Long, EventT>>>, ContiguousSequenceRange> + sideInputProducer, + Duration globalSequenceGenerationFrequency) { + this.sideInputProducer = sideInputProducer; + this.frequencyOfGeneration = globalSequenceGenerationFrequency; + } + + @Override + public PCollectionView<ContiguousSequenceRange> expand( + PCollection<TimestampedValue<KV<EventKeyT, KV<Long, EventT>>>> 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.<TimestampedValue<KV<EventKeyT, KV<Long, EventT>>>>into( + (WindowFn<? super TimestampedValue<KV<EventKeyT, KV<Long, EventT>>>, ?>) + input.getWindowingStrategy().getWindowFn()) + .accumulatingFiredPanes() + // TODO: verify that we don't need to have the lateness parameterized + .withAllowedLateness(Duration.ZERO) + .triggering( + Repeatedly.forever( + AfterFirst.of( + AfterPane.elementCountAtLeast(1), Review Comment: If so, is there a point in emitting after processing time? ########## sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequenceTracker.java: ########## @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.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 <EventKeyT> type of event key + * @param <EventT> type of event + * @param <ResultT> type of processing result + * @param <StateT> type of state + */ +class GlobalSequenceTracker< + EventKeyT, EventT, ResultT, StateT extends MutableState<EventT, ResultT>> + extends PTransform< + PCollection<TimestampedValue<KV<EventKeyT, KV<Long, EventT>>>>, + PCollectionView<ContiguousSequenceRange>> { + + private final Combine.GloballyAsSingletonView< + TimestampedValue<KV<EventKeyT, KV<Long, EventT>>>, ContiguousSequenceRange> + sideInputProducer; + private final @Nullable Duration frequencyOfGeneration; + + public GlobalSequenceTracker( + Combine.GloballyAsSingletonView< + TimestampedValue<KV<EventKeyT, KV<Long, EventT>>>, ContiguousSequenceRange> + sideInputProducer) { + this.sideInputProducer = sideInputProducer; + this.frequencyOfGeneration = null; + } + + public GlobalSequenceTracker( + Combine.GloballyAsSingletonView< + TimestampedValue<KV<EventKeyT, KV<Long, EventT>>>, ContiguousSequenceRange> + sideInputProducer, + Duration globalSequenceGenerationFrequency) { + this.sideInputProducer = sideInputProducer; + this.frequencyOfGeneration = globalSequenceGenerationFrequency; + } + + @Override + public PCollectionView<ContiguousSequenceRange> expand( + PCollection<TimestampedValue<KV<EventKeyT, KV<Long, EventT>>>> 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.<TimestampedValue<KV<EventKeyT, KV<Long, EventT>>>>into( + (WindowFn<? super TimestampedValue<KV<EventKeyT, KV<Long, EventT>>>, ?>) + input.getWindowingStrategy().getWindowFn()) + .accumulatingFiredPanes() + // TODO: verify that we don't need to have the lateness parameterized + .withAllowedLateness(Duration.ZERO) + .triggering( + Repeatedly.forever( + AfterFirst.of( + AfterPane.elementCountAtLeast(1), Review Comment: Will this just cause this trigger to fire repeatedly after every element it sees? ########## sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java: ########## @@ -0,0 +1,272 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.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 <EventT> + * @param <EventKeyT> + * @param <ResultT> + * @param <StateT> + */ +class GlobalSequencesProcessorDoFn< + EventT, EventKeyT, ResultT, StateT extends MutableState<EventT, ResultT>> + extends ProcessorDoFn<EventT, EventKeyT, ResultT, StateT> { + + 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); Review Comment: Hm, noting that I think the usage of this timer will cause issues if the input has allowed lateness. We may need to disallow handling late data here -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
