reswqa commented on code in PR #25978: URL: https://github.com/apache/flink/pull/25978#discussion_r1916024201
########## flink-datastream-api/src/main/java/org/apache/flink/datastream/api/extension/eventtime/EventTimeExtension.java: ########## @@ -0,0 +1,207 @@ +/* + * 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.flink.datastream.api.extension.eventtime; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.watermark.BoolWatermarkDeclaration; +import org.apache.flink.api.common.watermark.LongWatermarkDeclaration; +import org.apache.flink.api.common.watermark.Watermark; +import org.apache.flink.api.common.watermark.WatermarkDeclarations; +import org.apache.flink.datastream.api.extension.eventtime.function.EventTimeProcessFunction; +import org.apache.flink.datastream.api.extension.eventtime.function.OneInputEventTimeStreamProcessFunction; +import org.apache.flink.datastream.api.extension.eventtime.function.TwoInputBroadcastEventTimeStreamProcessFunction; +import org.apache.flink.datastream.api.extension.eventtime.function.TwoInputNonBroadcastEventTimeStreamProcessFunction; +import org.apache.flink.datastream.api.extension.eventtime.function.TwoOutputEventTimeStreamProcessFunction; +import org.apache.flink.datastream.api.extension.eventtime.strategy.EventTimeExtractor; +import org.apache.flink.datastream.api.extension.eventtime.strategy.EventTimeWatermarkGeneratorBuilder; +import org.apache.flink.datastream.api.extension.eventtime.strategy.EventTimeWatermarkStrategy; +import org.apache.flink.datastream.api.extension.eventtime.timer.EventTimeManager; +import org.apache.flink.datastream.api.function.OneInputStreamProcessFunction; +import org.apache.flink.datastream.api.function.TwoOutputStreamProcessFunction; +import org.apache.flink.datastream.api.stream.KeyedPartitionStream; + +/** + * The entry point for the event-time extension, which provides the following functionality: + * + * <ul> + * <li>defines the event-time watermark. + * <li>provides the {@link EventTimeWatermarkGeneratorBuilder} to facilitate the generation of + * event time watermarks. + * <li>provides a tool to encapsulate a user-defined {@link EventTimeProcessFunction} to provide + * the relevant components of the event-time extension. + * </ul> + */ +@Experimental +public class EventTimeExtension { + + private static final Class<?> INSTANCE; + + static { + try { + INSTANCE = + Class.forName( + "org.apache.flink.datastream.impl.extension.eventtime.EventTimeExtensionImpl"); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Please ensure that flink-datastream in your class path"); + } + } + + // =============== Event Time related Watermark Declarations =============== + + public static final LongWatermarkDeclaration EVENT_TIME_WATERMARK_DECLARATION = + WatermarkDeclarations.newBuilder("BUILTIN_API_EVENT_TIME") + .typeLong() + .combineFunctionMin() + .defaultHandlingStrategyForward() Review Comment: combineWaitForAllChannels ########## flink-datastream-api/src/main/java/org/apache/flink/datastream/api/extension/eventtime/EventTimeExtension.java: ########## @@ -0,0 +1,207 @@ +/* + * 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.flink.datastream.api.extension.eventtime; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.watermark.BoolWatermarkDeclaration; +import org.apache.flink.api.common.watermark.LongWatermarkDeclaration; +import org.apache.flink.api.common.watermark.Watermark; +import org.apache.flink.api.common.watermark.WatermarkDeclarations; +import org.apache.flink.datastream.api.extension.eventtime.function.EventTimeProcessFunction; +import org.apache.flink.datastream.api.extension.eventtime.function.OneInputEventTimeStreamProcessFunction; +import org.apache.flink.datastream.api.extension.eventtime.function.TwoInputBroadcastEventTimeStreamProcessFunction; +import org.apache.flink.datastream.api.extension.eventtime.function.TwoInputNonBroadcastEventTimeStreamProcessFunction; +import org.apache.flink.datastream.api.extension.eventtime.function.TwoOutputEventTimeStreamProcessFunction; +import org.apache.flink.datastream.api.extension.eventtime.strategy.EventTimeExtractor; +import org.apache.flink.datastream.api.extension.eventtime.strategy.EventTimeWatermarkGeneratorBuilder; +import org.apache.flink.datastream.api.extension.eventtime.strategy.EventTimeWatermarkStrategy; +import org.apache.flink.datastream.api.extension.eventtime.timer.EventTimeManager; +import org.apache.flink.datastream.api.function.OneInputStreamProcessFunction; +import org.apache.flink.datastream.api.function.TwoOutputStreamProcessFunction; +import org.apache.flink.datastream.api.stream.KeyedPartitionStream; + +/** + * The entry point for the event-time extension, which provides the following functionality: + * + * <ul> + * <li>defines the event-time watermark. + * <li>provides the {@link EventTimeWatermarkGeneratorBuilder} to facilitate the generation of + * event time watermarks. + * <li>provides a tool to encapsulate a user-defined {@link EventTimeProcessFunction} to provide + * the relevant components of the event-time extension. + * </ul> + */ +@Experimental +public class EventTimeExtension { + + private static final Class<?> INSTANCE; + + static { + try { + INSTANCE = + Class.forName( + "org.apache.flink.datastream.impl.extension.eventtime.EventTimeExtensionImpl"); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Please ensure that flink-datastream in your class path"); + } + } + + // =============== Event Time related Watermark Declarations =============== + + public static final LongWatermarkDeclaration EVENT_TIME_WATERMARK_DECLARATION = + WatermarkDeclarations.newBuilder("BUILTIN_API_EVENT_TIME") + .typeLong() + .combineFunctionMin() + .defaultHandlingStrategyForward() + .build(); + + public static final BoolWatermarkDeclaration IDLE_STATUS_WATERMARK_DECLARATION = + WatermarkDeclarations.newBuilder("BUILTIN_API_EVENT_TIME_IDLE") + .typeBool() + .combineFunctionAND() + .defaultHandlingStrategyForward() Review Comment: combineWaitForAllChannels ########## flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/extension/eventtime/EventTimeExtensionITCase.java: ########## @@ -0,0 +1,327 @@ +/* + * 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.flink.test.streaming.api.datastream.extension.eventtime; + +import org.apache.flink.api.common.watermark.LongWatermark; +import org.apache.flink.api.common.watermark.Watermark; +import org.apache.flink.api.common.watermark.WatermarkHandlingResult; +import org.apache.flink.api.connector.dsv2.DataStreamV2SourceUtils; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.datastream.api.ExecutionEnvironment; +import org.apache.flink.datastream.api.common.Collector; +import org.apache.flink.datastream.api.context.NonPartitionedContext; +import org.apache.flink.datastream.api.context.PartitionedContext; +import org.apache.flink.datastream.api.extension.eventtime.EventTimeExtension; +import org.apache.flink.datastream.api.extension.eventtime.function.OneInputEventTimeStreamProcessFunction; +import org.apache.flink.datastream.api.extension.eventtime.timer.EventTimeManager; +import org.apache.flink.datastream.api.function.OneInputStreamProcessFunction; +import org.apache.flink.datastream.api.stream.NonKeyedPartitionStream; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; + +import java.io.Serializable; +import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; + +/** This ITCase class tests the behavior of {@link EventTimeExtension}. */ +class EventTimeExtensionITCase implements Serializable { Review Comment: We should also introduce test cases for two input watermark alignment. ########## flink-runtime/src/main/java/org/apache/flink/streaming/util/watermark/WatermarkUtils.java: ########## @@ -95,4 +99,28 @@ private static Collection<? extends WatermarkDeclaration> getWatermarkDeclaratio .map(AbstractInternalWatermarkDeclaration::from) .collect(Collectors.toSet()); } + + /** Create watermark combiners if there are event time watermark declarations. */ + public static void addEventTimeWatermarkCombinerIfNeeded( + Set<AbstractInternalWatermarkDeclaration<?>> watermarkDeclarationSet, + Map<String, WatermarkCombiner> watermarkCombiners, + int numberOfInputChannels) { + Set<String> declaredWatermarkIdentifiers = + watermarkDeclarationSet.stream() + .map(AbstractInternalWatermarkDeclaration::getIdentifier) + .collect(Collectors.toSet()); + + // create event time watermark combiner + if (declaredWatermarkIdentifiers.contains( + EventTimeExtension.EVENT_TIME_WATERMARK_DECLARATION.getIdentifier())) { + EventTimeWatermarkCombiner eventTimeWatermarkCombiner = + new EventTimeWatermarkCombiner(numberOfInputChannels); + watermarkCombiners.put( + EventTimeExtension.EVENT_TIME_WATERMARK_DECLARATION.getIdentifier(), + eventTimeWatermarkCombiner); + watermarkCombiners.put( + EventTimeExtension.IDLE_STATUS_WATERMARK_DECLARATION.getIdentifier(), + eventTimeWatermarkCombiner); + } Review Comment: We can avoid the allocation. ```suggestion if (watermarkDeclarationSet.stream() .anyMatch( declaration -> EventTimeExtension.isEventTimeWatermark( declaration.getIdentifier()))) { EventTimeWatermarkCombiner eventTimeWatermarkCombiner = new EventTimeWatermarkCombiner(numberOfInputChannels); watermarkCombiners.put( EventTimeExtension.EVENT_TIME_WATERMARK_DECLARATION.getIdentifier(), eventTimeWatermarkCombiner); watermarkCombiners.put( EventTimeExtension.IDLE_STATUS_WATERMARK_DECLARATION.getIdentifier(), eventTimeWatermarkCombiner); } ``` ########## flink-datastream/src/main/java/org/apache/flink/datastream/impl/extension/eventtime/timer/DefaultEventTimeManager.java: ########## @@ -0,0 +1,72 @@ +/* + * 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.flink.datastream.impl.extension.eventtime.timer; + +import org.apache.flink.datastream.api.extension.eventtime.timer.EventTimeManager; +import org.apache.flink.runtime.state.VoidNamespace; +import org.apache.flink.streaming.api.operators.InternalTimerService; + +import javax.annotation.Nullable; + +import java.util.function.Supplier; + +/** The implementation of {@link EventTimeManager}. */ +public class DefaultEventTimeManager implements EventTimeManager { + + /** + * The timer service of operator, used in register event timer. Note that it cloud be null if + * the operator is not a keyed operator. + */ + @Nullable private final InternalTimerService<VoidNamespace> timerService; + + /** The supplier of the current event time. */ + private final Supplier<Long> eventTimeSupplier; + + public DefaultEventTimeManager( + @Nullable InternalTimerService<VoidNamespace> timerService, + Supplier<Long> eventTimeSupplier) { + this.timerService = timerService; + this.eventTimeSupplier = eventTimeSupplier; + } + + @Override + public void registerTimer(long timestamp) { + if (timerService == null) { + throw new UnsupportedOperationException( + "Registering event timer is not allowed in NonKeyed Stream."); + } + + timerService.registerEventTimeTimer(VoidNamespace.INSTANCE, timestamp); + } + + @Override + public void deleteTimer(long timestamp) { + if (timerService == null) { + throw new UnsupportedOperationException( + "Registering event timer is not allowed in NonKeyed Stream."); Review Comment: ```suggestion "Deleting event timer is not allowed in NonKeyed Stream."); ``` ########## flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/extension/eventtime/EventTimeExtensionITCase.java: ########## @@ -0,0 +1,327 @@ +/* + * 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.flink.test.streaming.api.datastream.extension.eventtime; + +import org.apache.flink.api.common.watermark.LongWatermark; +import org.apache.flink.api.common.watermark.Watermark; +import org.apache.flink.api.common.watermark.WatermarkHandlingResult; +import org.apache.flink.api.connector.dsv2.DataStreamV2SourceUtils; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.datastream.api.ExecutionEnvironment; +import org.apache.flink.datastream.api.common.Collector; +import org.apache.flink.datastream.api.context.NonPartitionedContext; +import org.apache.flink.datastream.api.context.PartitionedContext; +import org.apache.flink.datastream.api.extension.eventtime.EventTimeExtension; +import org.apache.flink.datastream.api.extension.eventtime.function.OneInputEventTimeStreamProcessFunction; +import org.apache.flink.datastream.api.extension.eventtime.timer.EventTimeManager; +import org.apache.flink.datastream.api.function.OneInputStreamProcessFunction; +import org.apache.flink.datastream.api.stream.NonKeyedPartitionStream; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; + +import java.io.Serializable; +import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; + +/** This ITCase class tests the behavior of {@link EventTimeExtension}. */ +class EventTimeExtensionITCase implements Serializable { + + private static ConcurrentLinkedQueue<Tuple2<Long, String>> outputRecords = + new ConcurrentLinkedQueue<>(); + private static ConcurrentLinkedQueue<Long> outputEventTimes = new ConcurrentLinkedQueue<>(); + private static ConcurrentLinkedQueue<Long> invokedTimerTimes = new ConcurrentLinkedQueue<>(); Review Comment: 1. Why this field be static? 2. Can be final. -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org