From a4013fc971f52d7216941aeb6a914293fae4038f Mon Sep 17 00:00:00 2001 From: sxnan Date: Fri, 11 Aug 2023 15:16:43 +0800 Subject: [PATCH] [FLINK-33202][runtime] Support switching from batch to stream mode to improve throughput when processing backlog data --- .../source/reader/SourceReaderBaseTest.java | 4 + .../source/coordinator/SourceCoordinator.java | 9 + .../coordinator/SourceCoordinatorContext.java | 16 + .../runtime/source/event/BacklogEvent.java | 34 ++ .../streaming/api/graph/StreamConfig.java | 14 +- .../api/graph/StreamGraphGenerator.java | 8 + .../api/operators/AbstractStreamOperator.java | 58 ++++ .../api/operators/BacklogTimeService.java | 91 +++++ .../api/operators/CountingOutput.java | 6 + .../flink/streaming/api/operators/Input.java | 9 + .../InternalBacklogAwareTimerServiceImpl.java | 138 ++++++++ ...alBacklogAwareTimerServiceManagerImpl.java | 161 +++++++++ .../InternalTimeServiceManagerImpl.java | 4 +- .../operators/InternalTimerServiceImpl.java | 4 + .../api/operators/OperatorAttributes.java | 55 +++ .../operators/OperatorAttributesBuilder.java | 64 ++++ .../flink/streaming/api/operators/Output.java | 11 + .../api/operators/SourceOperator.java | 8 + .../api/operators/StreamOperator.java | 6 + .../api/operators/StreamOperatorFactory.java | 6 + .../api/operators/TwoInputStreamOperator.java | 16 + .../sort/MultiInputSortingDataInput.java | 4 + .../sort/SortingBacklogDataInput.java | 323 ++++++++++++++++++ .../api/operators/sort/SortingDataInput.java | 4 + .../BatchExecutionInternalTimeService.java | 35 +- .../ProgressiveTimestampsAndWatermarks.java | 2 +- .../source/TimestampsAndWatermarks.java | 2 + .../AbstractMultipleInputTransformation.java | 5 + .../OneInputTransformation.java | 5 + .../PhysicalTransformation.java | 4 + .../TwoInputTransformation.java | 5 + .../io/AbstractStreamTaskNetworkInput.java | 41 ++- .../runtime/io/FinishedDataOutput.java | 6 + .../runtime/io/PushingAsyncDataInput.java | 3 + .../runtime/io/RecordAttributesValve.java | 95 ++++++ .../runtime/io/RecordWriterOutput.java | 11 + .../StreamMultipleInputProcessorFactory.java | 6 + .../io/StreamTwoInputProcessorFactory.java | 10 + .../InternalRecordAttributes.java | 67 ++++ .../streamrecord/RecordAttributes.java | 66 ++++ .../streamrecord/RecordAttributesBuilder.java | 77 +++++ .../runtime/streamrecord/StreamElement.java | 22 +- .../streamrecord/StreamElementSerializer.java | 10 + .../tasks/BroadcastingOutputCollector.java | 8 + .../runtime/tasks/ChainingOutput.java | 10 + .../runtime/tasks/OneInputStreamTask.java | 30 ++ .../tasks/SourceOperatorStreamTask.java | 6 + .../runtime/translators/BacklogUtils.java | 68 ++++ .../MultiInputTransformationTranslator.java | 3 +- .../OneInputTransformationTranslator.java | 29 +- .../ReduceTransformationTranslator.java | 31 +- .../TwoInputTransformationTranslator.java | 2 + ...bstractUdfStreamOperatorLifecycleTest.java | 1 + .../api/operators/BacklogTimeServiceTest.java | 75 ++++ ...ernalBacklogAwareTimerServiceImplTest.java | 111 ++++++ .../InternalTimerServiceImplTest.java | 39 +-- .../api/operators/TestKeyContext.java | 34 ++ .../streaming/api/operators/TestTrigger.java | 64 ++++ .../api/operators/TimeServiceTestUtils.java | 81 +++++ .../operators/sort/CollectingDataOutput.java | 6 + .../operators/sort/CollectionDataInput.java | 2 + .../sort/LargeSortingDataInputITCase.java | 4 + .../sort/SortingBacklogDataInputTest.java | 235 +++++++++++++ ...BatchExecutionInternalTimeServiceTest.java | 51 +-- .../source/CollectingDataOutput.java | 6 + .../runtime/io/RecordAttributesValveTest.java | 58 ++++ .../io/StreamTaskNetworkInputTest.java | 4 + .../StreamElementSerializerTest.java | 5 + .../StatusWatermarkValveTest.java | 6 + .../util/SourceOperatorTestHarness.java | 6 + .../planner/runtime/utils/TimeTestUtil.scala | 4 +- .../output/BroadcastingOutput.java | 8 + .../StreamingWithBacklogITCase.java | 265 ++++++++++++++ 73 files changed, 2647 insertions(+), 130 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/source/event/BacklogEvent.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/BacklogTimeService.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalBacklogAwareTimerServiceImpl.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalBacklogAwareTimerServiceManagerImpl.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorAttributes.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorAttributesBuilder.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sort/SortingBacklogDataInput.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordAttributesValve.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/InternalRecordAttributes.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributes.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributesBuilder.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/BacklogUtils.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/BacklogTimeServiceTest.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/InternalBacklogAwareTimerServiceImplTest.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TestKeyContext.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TestTrigger.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TimeServiceTestUtils.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/sort/SortingBacklogDataInputTest.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/RecordAttributesValveTest.java create mode 100644 flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/StreamingWithBacklogITCase.java diff --git a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/SourceReaderBaseTest.java b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/SourceReaderBaseTest.java index f5882d667978e0..517c624b4803c6 100644 --- a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/SourceReaderBaseTest.java +++ b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/SourceReaderBaseTest.java @@ -46,6 +46,7 @@ import org.apache.flink.streaming.api.operators.SourceOperator; import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; @@ -561,5 +562,8 @@ public void emitWatermarkStatus(WatermarkStatus watermarkStatus) {} @Override public void emitLatencyMarker(LatencyMarker latencyMarker) {} + + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) throws Exception {} } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinator.java index faeac9a8dc4973..386b66d6341de6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinator.java @@ -33,6 +33,7 @@ Licensed to the Apache Software Foundation (ASF) under one import org.apache.flink.runtime.operators.coordination.CoordinatorStore; import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.runtime.source.event.BacklogEvent; import org.apache.flink.runtime.source.event.ReaderRegistrationEvent; import org.apache.flink.runtime.source.event.ReportedWatermarkEvent; import org.apache.flink.runtime.source.event.RequestSplitEvent; @@ -607,6 +608,14 @@ private void handleReaderRegistrationEvent( context.registerSourceReader(subtask, attemptNumber, event.location()); if (!subtaskReaderExisted) { enumerator.addReader(event.subtaskId()); + + if (context.isBacklog() != null) { + context.runInCoordinatorThread( + () -> { + context.sendEventToSourceOperatorIfTaskReady( + subtask, new BacklogEvent(context.isBacklog())); + }); + } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java index 492cdf19e28325..7095ebd9c26ae9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java @@ -35,6 +35,7 @@ Licensed to the Apache Software Foundation (ASF) under one import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; import org.apache.flink.runtime.operators.coordination.OperatorEvent; import org.apache.flink.runtime.source.event.AddSplitEvent; +import org.apache.flink.runtime.source.event.BacklogEvent; import org.apache.flink.runtime.source.event.NoMoreSplitsEvent; import org.apache.flink.runtime.source.event.SourceEventWrapper; import org.apache.flink.util.ExceptionUtils; @@ -112,6 +113,7 @@ public class SourceCoordinatorContext private final boolean supportsConcurrentExecutionAttempts; private final boolean[] subtaskHasNoMoreSplits; private volatile boolean closed; + private volatile Boolean backlog = null; public SourceCoordinatorContext( SourceCoordinatorProvider.CoordinatorExecutorThreadFactory coordinatorThreadFactory, @@ -370,6 +372,16 @@ public void setIsProcessingBacklog(boolean isProcessingBacklog) { if (checkpointCoordinator != null) { checkpointCoordinator.setIsProcessingBacklog(operatorID, isProcessingBacklog); } + backlog = isProcessingBacklog; + callInCoordinatorThread( + () -> { + final BacklogEvent backlogEvent = new BacklogEvent(isProcessingBacklog); + for (int i = 0; i < getCoordinatorContext().currentParallelism(); i++) { + sendEventToSourceOperatorIfTaskReady(i, backlogEvent); + } + return null; + }, + "Failed to send BacklogEvent to reader."); } // --------- Package private additional methods for the SourceCoordinator ------------ @@ -629,6 +641,10 @@ private void sendCachedSplitsToNewlyRegisteredReader(int subtaskIndex, int attem } } + public Boolean isBacklog() { + return backlog; + } + /** Maintains the subtask gateways for different execution attempts of different subtasks. */ private static class SubtaskGateways { private final Map[] gateways; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/source/event/BacklogEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/source/event/BacklogEvent.java new file mode 100644 index 00000000000000..c2a8910974527f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/source/event/BacklogEvent.java @@ -0,0 +1,34 @@ +/* + * 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.runtime.source.event; + +import org.apache.flink.runtime.operators.coordination.OperatorEvent; + +/** A source event that notify the source operator of the backlog status. */ +public class BacklogEvent implements OperatorEvent { + + private final boolean backlog; + + public BacklogEvent(boolean backlog) { + this.backlog = backlog; + } + + public boolean isBacklog() { + return backlog; + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java index 267289c181f305..7eaa7e7113f525 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java @@ -800,7 +800,13 @@ public enum InputRequirement { * records from {@link #SORTED} inputs. There are no guarantees on ordering between and * within the different {@link #PASS_THROUGH} inputs. */ - PASS_THROUGH; + PASS_THROUGH, + + /** + * Backlog records are grouped (sorted) by key and then fed to the operator one group at a + * time. Non-backlog records are passed to the operator directly. + */ + SORTED_DURING_BACKLOG; } /** Interface representing chained inputs. */ @@ -879,4 +885,10 @@ public static boolean requiresSorting(StreamConfig.InputConfig inputConfig) { && ((StreamConfig.NetworkInputConfig) inputConfig).getInputRequirement() == StreamConfig.InputRequirement.SORTED; } + + public static boolean requiresSortingDuringBacklog(StreamConfig.InputConfig inputConfig) { + return inputConfig instanceof StreamConfig.NetworkInputConfig + && ((StreamConfig.NetworkInputConfig) inputConfig).getInputRequirement() + == InputRequirement.SORTED_DURING_BACKLOG; + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java index 26a2ef6ede3867..770c6622e3b2f8 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java @@ -47,6 +47,7 @@ import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; +import org.apache.flink.streaming.api.operators.InternalBacklogAwareTimerServiceManagerImpl; import org.apache.flink.streaming.api.operators.sorted.state.BatchExecutionCheckpointStorage; import org.apache.flink.streaming.api.operators.sorted.state.BatchExecutionInternalTimeServiceManager; import org.apache.flink.streaming.api.operators.sorted.state.BatchExecutionStateBackend; @@ -93,6 +94,7 @@ import javax.annotation.Nullable; +import java.time.Duration; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -415,6 +417,12 @@ private void configureStreamGraphStreaming(final StreamGraph graph) { graph.setCheckpointStorage(checkpointStorage); graph.setSavepointDirectory(savepointDir); graph.setGlobalStreamExchangeMode(deriveGlobalStreamExchangeModeStreaming()); + + if (Duration.ZERO.equals( + configuration.get( + ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL_DURING_BACKLOG))) { + graph.setTimerServiceProvider(InternalBacklogAwareTimerServiceManagerImpl::create); + } } private String deriveJobName(String defaultJobName) { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java index 78fb35af4e0e3c..4bae4a3877b28d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.api.operators; +import org.apache.flink.annotation.Experimental; import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.VisibleForTesting; @@ -49,7 +50,10 @@ import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.CheckpointedStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.InternalRecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributesBuilder; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.StreamTask; @@ -61,6 +65,9 @@ import org.slf4j.LoggerFactory; import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; import java.util.Locale; import java.util.Optional; @@ -148,6 +155,9 @@ public abstract class AbstractStreamOperator protected transient ProcessingTimeService processingTimeService; + protected transient RecordAttributes lastRecordAttributes1; + protected transient RecordAttributes lastRecordAttributes2; + // ------------------------------------------------------------------------ // Life Cycle // ------------------------------------------------------------------------ @@ -649,4 +659,52 @@ public OperatorID getOperatorID() { protected Optional> getTimeServiceManager() { return Optional.ofNullable(timeServiceManager); } + + @Experimental + public void processRecordAttributes(RecordAttributes recordAttributes) throws Exception { + lastRecordAttributes1 = recordAttributes; + if (timeServiceManager != null + && timeServiceManager instanceof InternalBacklogAwareTimerServiceManagerImpl) { + final InternalBacklogAwareTimerServiceManagerImpl backlogAwareTimerServiceManager = + (InternalBacklogAwareTimerServiceManagerImpl) timeServiceManager; + if (recordAttributes instanceof InternalRecordAttributes) { + backlogAwareTimerServiceManager.setMaxWatermarkDuringBacklog( + ((InternalRecordAttributes) recordAttributes) + .getMaxWatermarkDuringBacklog()); + } + backlogAwareTimerServiceManager.setBacklog(recordAttributes.isBacklog()); + } + output.emitRecordAttributes( + new RecordAttributesBuilder(Collections.singletonList(recordAttributes)).build()); + } + + @Experimental + public void processRecordAttributes1(RecordAttributes recordAttributes) throws Exception { + lastRecordAttributes1 = recordAttributes; + List lastRecordAttributes = getTwoInputsLastRecordAttributes(); + output.emitRecordAttributes(new RecordAttributesBuilder(lastRecordAttributes).build()); + } + + @Experimental + public void processRecordAttributes2(RecordAttributes recordAttributes) throws Exception { + lastRecordAttributes2 = recordAttributes; + List lastRecordAttributes = getTwoInputsLastRecordAttributes(); + output.emitRecordAttributes(new RecordAttributesBuilder(lastRecordAttributes).build()); + } + + private List getTwoInputsLastRecordAttributes() { + Preconditions.checkState( + lastRecordAttributes1 != null || lastRecordAttributes2 != null, + "lastRecordAttributes1 and lastRecordAttributes2 cannot be both null."); + + List lastRecordAttributes; + if (lastRecordAttributes1 != null && lastRecordAttributes2 != null) { + lastRecordAttributes = Arrays.asList(lastRecordAttributes1, lastRecordAttributes2); + } else if (lastRecordAttributes1 != null) { + lastRecordAttributes = Collections.singletonList(lastRecordAttributes1); + } else { + lastRecordAttributes = Collections.singletonList(lastRecordAttributes2); + } + return lastRecordAttributes; + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/BacklogTimeService.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/BacklogTimeService.java new file mode 100644 index 00000000000000..1fd782b13b60cb --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/BacklogTimeService.java @@ -0,0 +1,91 @@ +/* + * 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.streaming.api.operators; + +import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue; +import org.apache.flink.streaming.api.operators.sorted.state.BatchExecutionInternalTimeService; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; + +import java.util.LinkedList; +import java.util.List; + +/** + * An implementation of a {@link InternalTimerService} that manages timers with a single active key + * at a time. This is used by {@link + * org.apache.flink.streaming.api.operators.InternalBacklogAwareTimerServiceImpl} during backlog + * processing. + */ +class BacklogTimeService extends BatchExecutionInternalTimeService { + private long maxWatermarkDuringBacklog; + + public BacklogTimeService( + ProcessingTimeService processingTimeService, + Triggerable triggerTarget, + KeyGroupedInternalPriorityQueue> eventTimeTimersQueue) { + super(processingTimeService, triggerTarget, eventTimeTimersQueue, null); + } + + @Override + public void registerProcessingTimeTimer(N namespace, long time) { + throw new UnsupportedOperationException( + "BacklogTimeService does not support registering processing timer."); + } + + @Override + public void deleteProcessingTimeTimer(N namespace, long time) { + throw new UnsupportedOperationException( + "BacklogTimeService does not support deleting processing timer."); + } + + /** + * Set the current key of the time service. If the new key is different from the last key, all + * the event time timers of the last key whose timestamp is less than or equal to the max + * watermark during backlog are triggered. + */ + public void setCurrentKey(K newKey) throws Exception { + if (newKey != null && newKey.equals(currentKey)) { + return; + } + + TimerHeapInternalTimer timer; + List> skippedTimers = new LinkedList<>(); + if (currentKey != null) { + while ((timer = eventTimeTimersQueue.peek()) != null + && timer.getTimestamp() <= maxWatermarkDuringBacklog) { + eventTimeTimersQueue.poll(); + + if (timer.getKey() != currentKey) { + skippedTimers.add(timer); + } else { + triggerTarget.onEventTime(timer); + } + } + eventTimeTimersQueue.addAll(skippedTimers); + } + + if (newKey == null) { + currentWatermark = maxWatermarkDuringBacklog; + } + + currentKey = newKey; + } + + public void setMaxWatermarkDuringBacklog(long watermark) { + maxWatermarkDuringBacklog = watermark; + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/CountingOutput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/CountingOutput.java index 96028c139a69f1..824f8ebebcac5e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/CountingOutput.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/CountingOutput.java @@ -22,6 +22,7 @@ import org.apache.flink.metrics.Gauge; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.WatermarkGaugeExposingOutput; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; @@ -74,4 +75,9 @@ public void close() { public Gauge getWatermarkGauge() { return output.getWatermarkGauge(); } + + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) { + output.emitRecordAttributes(recordAttributes); + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/Input.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/Input.java index d6ca8e6186a3d8..784db9db322265 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/Input.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/Input.java @@ -18,9 +18,11 @@ package org.apache.flink.streaming.api.operators; +import org.apache.flink.annotation.Experimental; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; @@ -69,4 +71,11 @@ public interface Input { * guaranteed to not be called concurrently with other methods of the operator. */ void setKeyContextElement(StreamRecord record) throws Exception; + + /** + * Processes a {@link RecordAttributes} that arrived at this input. This method is guaranteed to + * not be called concurrently with other methods of the operator. + */ + @Experimental + default void processRecordAttributes(RecordAttributes recordAttributes) throws Exception {} } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalBacklogAwareTimerServiceImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalBacklogAwareTimerServiceImpl.java new file mode 100644 index 00000000000000..5e72e480cd293c --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalBacklogAwareTimerServiceImpl.java @@ -0,0 +1,138 @@ +/* + * 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.streaming.api.operators; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.util.function.BiConsumerWithException; + +/** + * InternalBacklogAwareTimerServiceImpl is the implementation of InternalTimerService that manages + * event time timers differently under different backlog status. It manages the processing time + * timers in the same way as the {@link InternalTimerServiceImpl}. + * + *

The backlog status is false at the beginning. When backlog status is false, the timer services + * is the same as the {@link InternalTimerServiceImpl}, where event time timers are triggered as the + * watermark advancing. + * + *

When backlog status switch to true, the timer service manages event time timer with a single + * active key at a time. When the active key changed, all the event time timers of the last key are + * triggered up to the maximum watermark during backlog processing. + * + *

When the backlog status switch from true to false, all the event time timers of the last key + * are triggered up to the maximum watermark during backlog processing. And the current watermark of + * the timer service will be the maximum watermark during backlog processing. + */ +@Internal +public class InternalBacklogAwareTimerServiceImpl implements InternalTimerService { + + private final InternalTimerServiceImpl realTimeInternalTimeService; + private final BacklogTimeService backlogTimeService; + private InternalTimerService currentInternalTimerService; + + public InternalBacklogAwareTimerServiceImpl( + InternalTimerServiceImpl realTimeInternalTimeService, + BacklogTimeService backlogTimeService) { + this.realTimeInternalTimeService = realTimeInternalTimeService; + this.backlogTimeService = backlogTimeService; + this.currentInternalTimerService = realTimeInternalTimeService; + } + + @Override + public long currentProcessingTime() { + return realTimeInternalTimeService.currentProcessingTime(); + } + + @Override + public long currentWatermark() { + return currentInternalTimerService.currentWatermark(); + } + + @Override + public void registerProcessingTimeTimer(N namespace, long time) { + realTimeInternalTimeService.registerProcessingTimeTimer(namespace, time); + } + + @Override + public void deleteProcessingTimeTimer(N namespace, long time) { + realTimeInternalTimeService.deleteProcessingTimeTimer(namespace, time); + } + + @Override + public void registerEventTimeTimer(N namespace, long time) { + currentInternalTimerService.registerEventTimeTimer(namespace, time); + } + + @Override + public void deleteEventTimeTimer(N namespace, long time) { + currentInternalTimerService.deleteEventTimeTimer(namespace, time); + } + + @Override + public void forEachEventTimeTimer(BiConsumerWithException consumer) + throws Exception { + currentInternalTimerService.forEachEventTimeTimer(consumer); + } + + @Override + public void forEachProcessingTimeTimer(BiConsumerWithException consumer) + throws Exception { + realTimeInternalTimeService.forEachProcessingTimeTimer(consumer); + } + + /** + * Advances the Watermark of the {@link InternalTimerServiceImpl} during real time processing, + * potentially firing event time timers. + */ + public void advanceWatermark(long timestamp) throws Exception { + realTimeInternalTimeService.advanceWatermark(timestamp); + } + + /** + * Set the maximum watermark during backlog of the {@link InternalBacklogAwareTimerServiceImpl}. + */ + public void setMaxWatermarkDuringBacklog(long timestamp) { + backlogTimeService.setMaxWatermarkDuringBacklog(timestamp); + } + + /** Set the backlog status of the timer service. */ + public void setBacklog(boolean backlog) throws Exception { + if (currentInternalTimerService == backlogTimeService && !backlog) { + // Switch to non backlog + backlogTimeService.setCurrentKey(null); + currentInternalTimerService = realTimeInternalTimeService; + realTimeInternalTimeService.advanceWatermark(backlogTimeService.currentWatermark()); + return; + } + + if (currentInternalTimerService == realTimeInternalTimeService && backlog) { + // Switch to backlog + currentInternalTimerService = backlogTimeService; + } + } + + /** + * Set the current key of the {@link InternalBacklogAwareTimerServiceImpl} during backlog + * processing. + */ + public void setCurrentKey(K newKey) throws Exception { + if (currentInternalTimerService != backlogTimeService) { + return; + } + backlogTimeService.setCurrentKey(newKey); + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalBacklogAwareTimerServiceManagerImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalBacklogAwareTimerServiceManagerImpl.java new file mode 100644 index 00000000000000..b0f352c6330e07 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalBacklogAwareTimerServiceManagerImpl.java @@ -0,0 +1,161 @@ +/* + * 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.streaming.api.operators; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.state.CheckpointableKeyedStateBackend; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider; +import org.apache.flink.runtime.state.KeyedStateBackend; +import org.apache.flink.runtime.state.KeyedStateCheckpointOutputStream; +import org.apache.flink.runtime.state.PriorityQueueSetFactory; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.StreamTaskCancellationContext; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.WrappingRuntimeException; + +import java.util.HashMap; +import java.util.Map; + +/** + * InternalBacklogAwareTimerServiceManagerImpl keeps track of all the {@link + * InternalBacklogAwareTimerServiceImpl}. + */ +@Internal +public class InternalBacklogAwareTimerServiceManagerImpl + extends InternalTimeServiceManagerImpl + implements InternalTimeServiceManager, KeyedStateBackend.KeySelectionListener { + + private final Map> timerServices = + new HashMap<>(); + + private boolean backlog = false; + + InternalBacklogAwareTimerServiceManagerImpl( + KeyGroupRange localKeyGroupRange, + KeyContext keyContext, + PriorityQueueSetFactory priorityQueueSetFactory, + ProcessingTimeService processingTimeService, + StreamTaskCancellationContext cancellationContext) { + super( + localKeyGroupRange, + keyContext, + priorityQueueSetFactory, + processingTimeService, + cancellationContext); + } + + @Override + @SuppressWarnings("unchecked") + public InternalTimerService getInternalTimerService( + String name, + TypeSerializer keySerializer, + TypeSerializer namespaceSerializer, + Triggerable triggerable) { + InternalBacklogAwareTimerServiceImpl timerService = + (InternalBacklogAwareTimerServiceImpl) timerServices.get(name); + if (timerService == null) { + final InternalTimerServiceImpl internalTimerService = + (InternalTimerServiceImpl) + super.getInternalTimerService( + name, keySerializer, namespaceSerializer, triggerable); + final BacklogTimeService backlogTimeService = + new BacklogTimeService<>( + processingTimeService, + triggerable, + internalTimerService.getEventTimeTimersQueue()); + timerService = + new InternalBacklogAwareTimerServiceImpl<>( + internalTimerService, backlogTimeService); + timerServices.put(name, timerService); + } + return timerService; + } + + @Override + public void advanceWatermark(Watermark watermark) throws Exception { + for (InternalBacklogAwareTimerServiceImpl service : timerServices.values()) { + service.advanceWatermark(watermark.getTimestamp()); + } + } + + @Override + public void snapshotToRawKeyedState( + KeyedStateCheckpointOutputStream stateCheckpointOutputStream, String operatorName) + throws Exception { + Preconditions.checkState(!backlog, "Cannot snapshot state during backlog."); + super.snapshotToRawKeyedState(stateCheckpointOutputStream, operatorName); + } + + public static InternalBacklogAwareTimerServiceManagerImpl create( + CheckpointableKeyedStateBackend keyedStateBackend, + ClassLoader userClassloader, + KeyContext keyContext, + ProcessingTimeService processingTimeService, + Iterable rawKeyedStates, + StreamTaskCancellationContext cancellationContext) + throws Exception { + + final InternalBacklogAwareTimerServiceManagerImpl manager = + new InternalBacklogAwareTimerServiceManagerImpl<>( + keyedStateBackend.getKeyGroupRange(), + keyContext, + keyedStateBackend, + processingTimeService, + cancellationContext); + + keyedStateBackend.registerKeySelectionListener(manager); + + return manager; + } + + @Override + public void keySelected(K newKey) { + try { + for (InternalBacklogAwareTimerServiceImpl value : timerServices.values()) { + value.setCurrentKey(newKey); + } + } catch (Exception e) { + throw new WrappingRuntimeException(e); + } + } + + /** + * Set the maximum watermark during backlog of all the managed {@link + * InternalBacklogAwareTimerServiceImpl}. + */ + public void setMaxWatermarkDuringBacklog(long timestamp) { + for (InternalBacklogAwareTimerServiceImpl timerService : timerServices.values()) { + timerService.setMaxWatermarkDuringBacklog(timestamp); + } + } + + /** Set the backlog status all the managed {@link InternalBacklogAwareTimerServiceImpl}. */ + public void setBacklog(boolean backlog) { + try { + for (InternalBacklogAwareTimerServiceImpl value : timerServices.values()) { + value.setBacklog(backlog); + } + this.backlog = backlog; + } catch (Exception e) { + throw new WrappingRuntimeException(e); + } + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManagerImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManagerImpl.java index 51a280bdda2bcb..f31ca166635516 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManagerImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManagerImpl.java @@ -69,12 +69,12 @@ public class InternalTimeServiceManagerImpl implements InternalTimeServiceMan private final KeyGroupRange localKeyGroupRange; private final KeyContext keyContext; private final PriorityQueueSetFactory priorityQueueSetFactory; - private final ProcessingTimeService processingTimeService; + protected final ProcessingTimeService processingTimeService; private final StreamTaskCancellationContext cancellationContext; private final Map> timerServices; - private InternalTimeServiceManagerImpl( + protected InternalTimeServiceManagerImpl( KeyGroupRange localKeyGroupRange, KeyContext keyContext, PriorityQueueSetFactory priorityQueueSetFactory, diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java index e2c7e4139b2a4c..dbc5c41894925b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java @@ -396,6 +396,10 @@ public int numEventTimeTimers(N namespace) { return countTimersInNamespaceInternal(namespace, eventTimeTimersQueue); } + public KeyGroupedInternalPriorityQueue> getEventTimeTimersQueue() { + return eventTimeTimersQueue; + } + private int countTimersInNamespaceInternal( N namespace, InternalPriorityQueue> queue) { int count = 0; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorAttributes.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorAttributes.java new file mode 100644 index 00000000000000..5889f92af81374 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorAttributes.java @@ -0,0 +1,55 @@ +/* + * 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.streaming.api.operators; + +import org.apache.flink.annotation.Experimental; + +/** + * OperatorAttributes element provides Job Manager with information that can be used to optimize job + * performance. + */ +@Experimental +public class OperatorAttributes { + + private final boolean internalSorterSupported; + + public OperatorAttributes(boolean internalSorterSupported) { + this.internalSorterSupported = internalSorterSupported; + } + + /** + * Returns true iff the operator uses an internal sorter to sort inputs by key when any of the + * following conditions are met: + * + *

    + *
  • execution.runtime-mode = BATCH. + *
  • execution.checkpointing.interval-during-backlog = 0 AND any of its input has + * isBacklog=true. + *
+ * + *

Here are the implications when it is true: + * + *

    + *
  • Its input records will not to be sorted externally before being fed into this operator. + *
  • Its managed memory will be set according to execution.sorted-inputs.memory. + *
+ */ + public boolean isInternalSorterSupported() { + return internalSorterSupported; + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorAttributesBuilder.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorAttributesBuilder.java new file mode 100644 index 00000000000000..e461a625e452e0 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorAttributesBuilder.java @@ -0,0 +1,64 @@ +/* + * 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.streaming.api.operators; + +import org.apache.flink.annotation.Experimental; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +/** The builder class for {@link OperatorAttributes}. */ +@Experimental +public class OperatorAttributesBuilder { + + private static final Logger LOG = LoggerFactory.getLogger(OperatorAttributesBuilder.class); + + @Nullable private Boolean internalSorterSupported = null; + + public OperatorAttributesBuilder() {} + + public OperatorAttributesBuilder setInternalSorterSupported(boolean internalSorterSupported) { + this.internalSorterSupported = internalSorterSupported; + return this; + } + + /** + * If any operator attribute is null, we will log it at DEBUG level and use the following + * default values. + * + *
    + *
  • internalSorterSupported defaults to false + *
+ */ + public OperatorAttributes build() { + return new OperatorAttributes( + getAttributeOrDefaultValue( + internalSorterSupported, "internalSorterSupported", false)); + } + + private T getAttributeOrDefaultValue( + @Nullable T attribute, String attributeName, T defaultValue) { + if (attribute == null) { + LOG.debug("{} is not set, set it to default value {}.", attributeName, defaultValue); + return defaultValue; + } + return attribute; + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/Output.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/Output.java index cdbeff8a6b11c0..fb3093c4dc1e9c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/Output.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/Output.java @@ -18,9 +18,11 @@ package org.apache.flink.streaming.api.operators; +import org.apache.flink.annotation.Experimental; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import org.apache.flink.util.Collector; @@ -55,4 +57,13 @@ public interface Output extends Collector { void collect(OutputTag outputTag, StreamRecord record); void emitLatencyMarker(LatencyMarker latencyMarker); + + /** + * Emits a {@link RecordAttributes} from an operator. This element is broadcast to all + * downstream operators. + */ + @Experimental + default void emitRecordAttributes(RecordAttributes recordAttributes) { + throw new UnsupportedOperationException(); + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java index c4f624e443c97c..6ee4aac004d99a 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java @@ -41,6 +41,7 @@ import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; import org.apache.flink.runtime.operators.coordination.OperatorEventHandler; import org.apache.flink.runtime.source.event.AddSplitEvent; +import org.apache.flink.runtime.source.event.BacklogEvent; import org.apache.flink.runtime.source.event.NoMoreSplitsEvent; import org.apache.flink.runtime.source.event.ReaderRegistrationEvent; import org.apache.flink.runtime.source.event.ReportedWatermarkEvent; @@ -56,6 +57,7 @@ import org.apache.flink.streaming.runtime.io.DataInputStatus; import org.apache.flink.streaming.runtime.io.MultipleFuturesAvailabilityHelper; import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributesBuilder; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.StreamTask; @@ -569,6 +571,12 @@ public void handleOperatorEvent(OperatorEvent event) { sourceReader.handleSourceEvents(((SourceEventWrapper) event).getSourceEvent()); } else if (event instanceof NoMoreSplitsEvent) { sourceReader.notifyNoMoreSplits(); + } else if (event instanceof BacklogEvent) { + eventTimeLogic.triggerPeriodicEmit(System.currentTimeMillis()); + output.emitRecordAttributes( + new RecordAttributesBuilder(Collections.emptyList()) + .setBacklog(((BacklogEvent) event).isBacklog()) + .build()); } else { throw new IllegalStateException("Received unexpected operator event " + event); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java index 134b9129a284f9..253aca83e7827b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java @@ -17,6 +17,7 @@ package org.apache.flink.streaming.api.operators; +import org.apache.flink.annotation.Experimental; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.state.CheckpointListener; import org.apache.flink.metrics.groups.OperatorMetricGroup; @@ -137,6 +138,11 @@ OperatorSnapshotFutures snapshotState( /** Provides a context to initialize all state in the operator. */ void initializeState(StreamTaskStateInitializer streamTaskStateManager) throws Exception; + @Experimental + default OperatorAttributes getOperatorAttributes() { + return new OperatorAttributesBuilder().build(); + } + // ------------------------------------------------------------------------ // miscellaneous // ------------------------------------------------------------------------ diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorFactory.java index 410028fb0d5523..d7dc5b7479ce9a 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorFactory.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorFactory.java @@ -17,6 +17,7 @@ package org.apache.flink.streaming.api.operators; +import org.apache.flink.annotation.Experimental; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -87,4 +88,9 @@ default void setInputType(TypeInformation type, ExecutionConfig executionConf /** Returns the runtime class of the stream operator. */ Class getStreamOperatorClass(ClassLoader classLoader); + + @Experimental + default OperatorAttributes getOperatorAttributes() { + return new OperatorAttributesBuilder().build(); + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TwoInputStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TwoInputStreamOperator.java index 8cb5ca342dcb1a..f4c2e022d3f712 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TwoInputStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TwoInputStreamOperator.java @@ -18,9 +18,11 @@ package org.apache.flink.streaming.api.operators; +import org.apache.flink.annotation.Experimental; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; @@ -98,4 +100,18 @@ public interface TwoInputStreamOperator extends StreamOperator implements StreamTaskInput { + + private static final Logger LOG = LoggerFactory.getLogger(SortingBacklogDataInput.class); + + private final StreamTaskInput wrappedInput; + private final PushSorter>> sorter; + private final KeySelector keySelector; + private final TypeSerializer keySerializer; + private final DataOutputSerializer dataOutputSerializer; + private final SortingDataOutput sortingDataOutput; + private final StreamTask.CanEmitBatchOfRecordsChecker canEmitBatchOfRecords; + private MutableObjectIterator>> sortedInput = null; + private long watermarkSeenDuringBacklog = Long.MIN_VALUE; + + private volatile OperatingMode mode = OperatingMode.PROCESSING_REALTIME; + + private enum OperatingMode { + + // We are processing realtime data, all the input data pass through immediately. + PROCESSING_REALTIME, + + // We are processing backlog data, all the input data are put in the sorter. + SORTING_BACKLOG, + + // We are switching from backlog processing to realtime processing, the data input the + // sorter are flushing to the downstream. + FLUSHING_BACKLOG + } + + public SortingBacklogDataInput( + StreamTaskInput wrappedInput, + TypeSerializer typeSerializer, + TypeSerializer keySerializer, + KeySelector keySelector, + MemoryManager memoryManager, + IOManager ioManager, + boolean objectReuse, + double managedMemoryFraction, + Configuration taskManagerConfiguration, + TaskInvokable containingTask, + ExecutionConfig executionConfig, + StreamTask.CanEmitBatchOfRecordsChecker canEmitBatchOfRecords) { + try { + this.canEmitBatchOfRecords = canEmitBatchOfRecords; + this.sortingDataOutput = new SortingDataOutput(); + this.keySelector = keySelector; + this.keySerializer = keySerializer; + int keyLength = keySerializer.getLength(); + final TypeComparator>> comparator; + if (keyLength > 0) { + this.dataOutputSerializer = new DataOutputSerializer(keyLength); + comparator = new FixedLengthByteKeyComparator<>(keyLength); + } else { + this.dataOutputSerializer = new DataOutputSerializer(64); + comparator = new VariableLengthByteKeyComparator<>(); + } + KeyAndValueSerializer keyAndValueSerializer = + new KeyAndValueSerializer<>(typeSerializer, keyLength); + this.wrappedInput = wrappedInput; + this.sorter = + ExternalSorter.newBuilder( + memoryManager, + containingTask, + keyAndValueSerializer, + comparator, + executionConfig) + .memoryFraction(managedMemoryFraction) + .enableSpilling( + ioManager, + taskManagerConfiguration.get( + AlgorithmOptions.SORT_SPILLING_THRESHOLD)) + .maxNumFileHandles( + taskManagerConfiguration.get(AlgorithmOptions.SPILLING_MAX_FAN)) + .objectReuse(objectReuse) + .largeRecords( + taskManagerConfiguration.get( + AlgorithmOptions.USE_LARGE_RECORDS_HANDLER)) + .build(); + } catch (MemoryAllocationException e) { + throw new RuntimeException(e); + } + } + + @Override + public int getInputIndex() { + return wrappedInput.getInputIndex(); + } + + @Override + public CompletableFuture prepareSnapshot( + ChannelStateWriter channelStateWriter, long checkpointId) throws CheckpointException { + if (mode != OperatingMode.PROCESSING_REALTIME) { + throw new UnsupportedOperationException( + "Checkpoints are not supported during backlog."); + } + return wrappedInput.prepareSnapshot(channelStateWriter, checkpointId); + } + + @Override + public void close() throws IOException { + IOException ex = null; + try { + wrappedInput.close(); + } catch (IOException e) { + ex = ExceptionUtils.firstOrSuppressed(e, ex); + } + + try { + sorter.close(); + } catch (IOException e) { + ex = ExceptionUtils.firstOrSuppressed(e, ex); + } + + if (ex != null) { + throw ex; + } + } + + @Override + public DataInputStatus emitNext(DataOutput output) throws Exception { + LOG.debug("Emit next, current mode: {}", mode); + if (sortingDataOutput.innerOutput != output) { + sortingDataOutput.innerOutput = output; + } + + if (mode == OperatingMode.PROCESSING_REALTIME) { + return wrappedInput.emitNext(sortingDataOutput); + } + + if (mode == OperatingMode.SORTING_BACKLOG) { + return wrappedInput.emitNext(sortingDataOutput); + } + + if (mode == OperatingMode.FLUSHING_BACKLOG) { + while (true) { + final DataInputStatus status = emitNextSortedRecord(output); + if (status == DataInputStatus.MORE_AVAILABLE + && canEmitBatchOfRecords.check() + && mode == OperatingMode.FLUSHING_BACKLOG) { + continue; + } + return status; + } + } + + // Should never reach here + throw new RuntimeException(String.format("Unknown OperatingMode %s", mode)); + } + + @Nonnull + private DataInputStatus emitNextSortedRecord(DataOutput output) throws Exception { + Tuple2> next = sortedInput.next(); + if (next != null) { + output.emitRecord(next.f1); + } else { + // Finished flushing + mode = OperatingMode.PROCESSING_REALTIME; + + // Send backlog=false downstream + output.emitRecordAttributes( + new RecordAttributesBuilder(Collections.emptyList()).setBacklog(false).build()); + + if (watermarkSeenDuringBacklog > Long.MIN_VALUE) { + output.emitWatermark(new Watermark(watermarkSeenDuringBacklog)); + } + } + return DataInputStatus.MORE_AVAILABLE; + } + + @Override + public CompletableFuture getAvailableFuture() { + if (mode == OperatingMode.FLUSHING_BACKLOG) { + return AvailabilityProvider.AVAILABLE; + } else { + return wrappedInput.getAvailableFuture(); + } + } + + private class SortingDataOutput implements DataOutput { + + private DataOutput innerOutput; + + @Override + public void emitRecord(StreamRecord streamRecord) throws Exception { + LOG.debug("Emit record {}", streamRecord.getValue()); + if (mode == OperatingMode.PROCESSING_REALTIME) { + innerOutput.emitRecord(streamRecord); + return; + } + + if (mode == OperatingMode.SORTING_BACKLOG) { + K key = keySelector.getKey(streamRecord.getValue()); + + keySerializer.serialize(key, dataOutputSerializer); + byte[] serializedKey = dataOutputSerializer.getCopyOfBuffer(); + dataOutputSerializer.clear(); + + sorter.writeRecord(Tuple2.of(serializedKey, streamRecord)); + return; + } + + throw new RuntimeException(String.format("Unexpected StreamRecord during %s.", mode)); + } + + @Override + public void emitWatermark(Watermark watermark) throws Exception { + if (mode == OperatingMode.PROCESSING_REALTIME) { + innerOutput.emitWatermark(watermark); + } else { + watermarkSeenDuringBacklog = + Math.max(watermarkSeenDuringBacklog, watermark.getTimestamp()); + } + } + + @Override + public void emitWatermarkStatus(WatermarkStatus watermarkStatus) throws Exception { + if (mode == OperatingMode.PROCESSING_REALTIME) { + innerOutput.emitWatermarkStatus(watermarkStatus); + } + + // Ignore watermark status during backlog + } + + @Override + public void emitLatencyMarker(LatencyMarker latencyMarker) throws Exception { + if (mode == OperatingMode.PROCESSING_REALTIME) { + innerOutput.emitLatencyMarker(latencyMarker); + } + + // Ignore latency marker during backlog + } + + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) throws Exception { + LOG.debug("Emit record attributes {}", recordAttributes); + if (mode == OperatingMode.PROCESSING_REALTIME && recordAttributes.isBacklog()) { + // switch to backlog + mode = OperatingMode.SORTING_BACKLOG; + innerOutput.emitRecordAttributes(recordAttributes); + return; + } + + if (mode == OperatingMode.SORTING_BACKLOG && !recordAttributes.isBacklog()) { + innerOutput.emitRecordAttributes( + new InternalRecordAttributes(true, watermarkSeenDuringBacklog)); + sorter.finishReading(); + sortedInput = sorter.getIterator(); + mode = OperatingMode.FLUSHING_BACKLOG; + return; + } + + if (mode == OperatingMode.FLUSHING_BACKLOG) { + throw new RuntimeException( + "Should not receive record attribute while flushing backlog."); + } + } + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sort/SortingDataInput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sort/SortingDataInput.java index 088519a2dec516..97e664a3682312 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sort/SortingDataInput.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sort/SortingDataInput.java @@ -39,6 +39,7 @@ import org.apache.flink.streaming.runtime.io.DataInputStatus; import org.apache.flink.streaming.runtime.io.StreamTaskInput; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import org.apache.flink.util.ExceptionUtils; @@ -183,6 +184,9 @@ public void emitWatermarkStatus(WatermarkStatus watermarkStatus) {} @Override public void emitLatencyMarker(LatencyMarker latencyMarker) {} + + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) {} } @Override diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionInternalTimeService.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionInternalTimeService.java index de590ffc4e2d41..c2f67741503e0b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionInternalTimeService.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionInternalTimeService.java @@ -40,38 +40,47 @@ public class BatchExecutionInternalTimeService implements InternalTimerSer private static final Logger LOG = LoggerFactory.getLogger(BatchExecutionInternalTimeService.class); - private final ProcessingTimeService processingTimeService; + protected final ProcessingTimeService processingTimeService; /** Processing time timers that are currently in-flight. */ - private final KeyGroupedInternalPriorityQueue> + protected final KeyGroupedInternalPriorityQueue> processingTimeTimersQueue; /** Event time timers that are currently in-flight. */ - private final KeyGroupedInternalPriorityQueue> + protected final KeyGroupedInternalPriorityQueue> eventTimeTimersQueue; /** * The local event time, as denoted by the last received {@link * org.apache.flink.streaming.api.watermark.Watermark Watermark}. */ - private long currentWatermark = Long.MIN_VALUE; + protected long currentWatermark = Long.MIN_VALUE; - private final Triggerable triggerTarget; + protected final Triggerable triggerTarget; - private K currentKey; + protected K currentKey; BatchExecutionInternalTimeService( ProcessingTimeService processingTimeService, Triggerable triggerTarget) { + this( + processingTimeService, + triggerTarget, + new BatchExecutionInternalPriorityQueueSet<>( + PriorityComparator.forPriorityComparableObjects(), 128), + new BatchExecutionInternalPriorityQueueSet<>( + PriorityComparator.forPriorityComparableObjects(), 128)); + } + protected BatchExecutionInternalTimeService( + ProcessingTimeService processingTimeService, + Triggerable triggerTarget, + KeyGroupedInternalPriorityQueue> eventTimeTimersQueue, + KeyGroupedInternalPriorityQueue> + processingTimeTimersQueue) { this.processingTimeService = checkNotNull(processingTimeService); this.triggerTarget = checkNotNull(triggerTarget); - - this.processingTimeTimersQueue = - new BatchExecutionInternalPriorityQueueSet<>( - PriorityComparator.forPriorityComparableObjects(), 128); - this.eventTimeTimersQueue = - new BatchExecutionInternalPriorityQueueSet<>( - PriorityComparator.forPriorityComparableObjects(), 128); + this.eventTimeTimersQueue = eventTimeTimersQueue; + this.processingTimeTimersQueue = processingTimeTimersQueue; } @Override diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/source/ProgressiveTimestampsAndWatermarks.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/source/ProgressiveTimestampsAndWatermarks.java index 14e6b6a7eb98fd..115aa9096b7c06 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/source/ProgressiveTimestampsAndWatermarks.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/source/ProgressiveTimestampsAndWatermarks.java @@ -152,7 +152,7 @@ public void stopPeriodicWatermarkEmits() { } } - void triggerPeriodicEmit(@SuppressWarnings("unused") long wallClockTimestamp) { + public void triggerPeriodicEmit(@SuppressWarnings("unused") long wallClockTimestamp) { if (currentPerSplitOutputs != null) { currentPerSplitOutputs.emitPeriodicWatermark(); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/source/TimestampsAndWatermarks.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/source/TimestampsAndWatermarks.java index cd41ca9ecb4d2c..14cf6d39cd618e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/source/TimestampsAndWatermarks.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/source/TimestampsAndWatermarks.java @@ -80,6 +80,8 @@ ReaderOutput createMainOutput( /** Stops emitting periodic watermarks. */ void stopPeriodicWatermarkEmits(); + default void triggerPeriodicEmit(long wallClockTimestamp) {} + // ------------------------------------------------------------------------ // factories // ------------------------------------------------------------------------ diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/AbstractMultipleInputTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/AbstractMultipleInputTransformation.java index 65bbc53188f5b7..adf2ae12738363 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/AbstractMultipleInputTransformation.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/AbstractMultipleInputTransformation.java @@ -86,4 +86,9 @@ public List> getTransitivePredecessors() { public final void setChainingStrategy(ChainingStrategy strategy) { operatorFactory.setChainingStrategy(strategy); } + + @Override + public boolean isInternalSorterSupported() { + return operatorFactory.getOperatorAttributes().isInternalSorterSupported(); + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java index 0a3957130fcd0b..b57eb680bce1e1 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java @@ -184,4 +184,9 @@ public List> getInputs() { public final void setChainingStrategy(ChainingStrategy strategy) { operatorFactory.setChainingStrategy(strategy); } + + @Override + public boolean isInternalSorterSupported() { + return operatorFactory.getOperatorAttributes().isInternalSorterSupported(); + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/PhysicalTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/PhysicalTransformation.java index fa1869cd9fb688..8dd6f18cc04a91 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/PhysicalTransformation.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/PhysicalTransformation.java @@ -76,4 +76,8 @@ public void setSupportsConcurrentExecutionAttempts( boolean supportsConcurrentExecutionAttempts) { this.supportsConcurrentExecutionAttempts = supportsConcurrentExecutionAttempts; } + + public boolean isInternalSorterSupported() { + return false; + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java index 4c576170741dfa..f338a690e287d9 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java @@ -229,4 +229,9 @@ public List> getTransitivePredecessors() { public final void setChainingStrategy(ChainingStrategy strategy) { operatorFactory.setChainingStrategy(strategy); } + + @Override + public boolean isInternalSorterSupported() { + return operatorFactory.getOperatorAttributes().isInternalSorterSupported(); + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/AbstractStreamTaskNetworkInput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/AbstractStreamTaskNetworkInput.java index db5198bd7160eb..cf274dd18b5dc1 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/AbstractStreamTaskNetworkInput.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/AbstractStreamTaskNetworkInput.java @@ -60,6 +60,7 @@ public abstract class AbstractStreamTaskNetworkInput< protected final StatusWatermarkValve statusWatermarkValve; protected final int inputIndex; + private final RecordAttributesValve recordAttributesValve; private InputChannelInfo lastChannel = null; private R currentRecordDeserializer = null; @@ -87,6 +88,8 @@ public AbstractStreamTaskNetworkInput( this.inputIndex = inputIndex; this.recordDeserializers = checkNotNull(recordDeserializers); this.canEmitBatchOfRecords = checkNotNull(canEmitBatchOfRecords); + this.recordAttributesValve = + new RecordAttributesValve(checkpointedInputGate.getNumberOfInputChannels()); } @Override @@ -107,8 +110,9 @@ public DataInputStatus emitNext(DataOutput output) throws Exception { } if (result.isFullRecord()) { - processElement(deserializationDelegate.getInstance(), output); - if (canEmitBatchOfRecords.check()) { + final boolean canContinue = + processElement(deserializationDelegate.getInstance(), output); + if (canEmitBatchOfRecords.check() && canContinue) { continue; } return DataInputStatus.MORE_AVAILABLE; @@ -141,19 +145,34 @@ public DataInputStatus emitNext(DataOutput output) throws Exception { } } - private void processElement(StreamElement recordOrMark, DataOutput output) throws Exception { - if (recordOrMark.isRecord()) { - output.emitRecord(recordOrMark.asRecord()); - } else if (recordOrMark.isWatermark()) { + /** + * Process the given stream element and returns whether to continue processing the next stream + * element without returning from the emitNext method. + */ + private boolean processElement(StreamElement streamElement, DataOutput output) + throws Exception { + if (streamElement.isRecord()) { + output.emitRecord(streamElement.asRecord()); + return true; + } else if (streamElement.isWatermark()) { statusWatermarkValve.inputWatermark( - recordOrMark.asWatermark(), flattenedChannelIndices.get(lastChannel), output); - } else if (recordOrMark.isLatencyMarker()) { - output.emitLatencyMarker(recordOrMark.asLatencyMarker()); - } else if (recordOrMark.isWatermarkStatus()) { + streamElement.asWatermark(), flattenedChannelIndices.get(lastChannel), output); + return true; + } else if (streamElement.isLatencyMarker()) { + output.emitLatencyMarker(streamElement.asLatencyMarker()); + return true; + } else if (streamElement.isWatermarkStatus()) { statusWatermarkValve.inputWatermarkStatus( - recordOrMark.asWatermarkStatus(), + streamElement.asWatermarkStatus(), flattenedChannelIndices.get(lastChannel), output); + return true; + } else if (streamElement.isRecordAttributes()) { + recordAttributesValve.inputRecordAttributes( + streamElement.asRecordAttributes(), + flattenedChannelIndices.get(lastChannel), + output); + return false; } else { throw new UnsupportedOperationException("Unknown type of StreamElement"); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/FinishedDataOutput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/FinishedDataOutput.java index 54769fa2f8ada8..60dff2e04f57ec 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/FinishedDataOutput.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/FinishedDataOutput.java @@ -20,6 +20,7 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; @@ -53,4 +54,9 @@ public void emitWatermarkStatus(WatermarkStatus watermarkStatus) throws Exceptio public void emitLatencyMarker(LatencyMarker latencyMarker) throws Exception { LOG.debug("Unexpected latency marker after finish() received."); } + + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) throws Exception { + LOG.debug("Unexpected recordAttributes after finish() received."); + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/PushingAsyncDataInput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/PushingAsyncDataInput.java index 619c2d00954a4f..63be0726dd57f8 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/PushingAsyncDataInput.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/PushingAsyncDataInput.java @@ -22,6 +22,7 @@ import org.apache.flink.runtime.io.PullingAsyncDataInput; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; @@ -55,5 +56,7 @@ interface DataOutput { void emitWatermarkStatus(WatermarkStatus watermarkStatus) throws Exception; void emitLatencyMarker(LatencyMarker latencyMarker) throws Exception; + + void emitRecordAttributes(RecordAttributes recordAttributes) throws Exception; } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordAttributesValve.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordAttributesValve.java new file mode 100644 index 00000000000000..7c7b3c3ae3b144 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordAttributesValve.java @@ -0,0 +1,95 @@ +/* + * 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.streaming.runtime.io; + +import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput.DataOutput; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributesBuilder; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; + +/** + * RecordAttributesValve combine RecordAttributes from different input channels. If any of the input + * channels is non backlog, the combined RecordAttributes is non backlog. + * + *

Currently, we only support switching the backlog status from null to backlog and backlog to + * non-backlog. Switching from non-backlog to backlog is not support at the moment, and it will be + * ignored. + */ +public class RecordAttributesValve { + + private static final Logger LOG = LoggerFactory.getLogger(RecordAttributesValve.class); + + private final int numInputChannels; + private final RecordAttributes[] allChannelRecordAttributes; + + private int backlogChannelsCnt = 0; + private RecordAttributes lastOutputAttributes = null; + private boolean hasBeenBacklog; + + public RecordAttributesValve(int numInputChannels) { + this.numInputChannels = numInputChannels; + this.allChannelRecordAttributes = new RecordAttributes[numInputChannels]; + this.hasBeenBacklog = false; + } + + public void inputRecordAttributes( + RecordAttributes recordAttributes, int channelIdx, DataOutput output) + throws Exception { + LOG.debug("RecordAttributes: {} from channel idx: {}", recordAttributes, channelIdx); + RecordAttributes lastChannelRecordAttributes = allChannelRecordAttributes[channelIdx]; + allChannelRecordAttributes[channelIdx] = recordAttributes; + + if (lastChannelRecordAttributes == null) { + lastChannelRecordAttributes = + new RecordAttributesBuilder(Collections.emptyList()).build(); + } + + if (lastChannelRecordAttributes.isBacklog() == recordAttributes.isBacklog()) { + return; + } + + if (recordAttributes.isBacklog()) { + backlogChannelsCnt += 1; + } else { + backlogChannelsCnt -= 1; + } + + if (lastOutputAttributes == null && backlogChannelsCnt != numInputChannels) { + return; + } + + final RecordAttributesBuilder builder = + new RecordAttributesBuilder(Collections.emptyList()); + builder.setBacklog(backlogChannelsCnt >= numInputChannels); + final RecordAttributes outputAttribute = builder.build(); + if (lastOutputAttributes == null + || lastOutputAttributes.isBacklog() != outputAttribute.isBacklog()) { + if (lastOutputAttributes != null && !lastOutputAttributes.isBacklog()) { + LOG.warn( + "Switching from non-backlog to backlog is currently not supported. Backlog status remains."); + return; + } + lastOutputAttributes = outputAttribute; + output.emitRecordAttributes(outputAttribute); + } + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java index 44e41e921971d0..fdeec69efc6db3 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java @@ -31,6 +31,7 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.metrics.WatermarkGauge; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamElement; import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -213,4 +214,14 @@ public void close() { public Gauge getWatermarkGauge() { return watermarkGauge; } + + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) { + try { + serializationDelegate.setInstance(recordAttributes); + recordWriter.broadcastEmit(serializationDelegate); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessorFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessorFactory.java index ee504eb76cf0ac..807f03060b3c13 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessorFactory.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessorFactory.java @@ -44,6 +44,7 @@ import org.apache.flink.streaming.runtime.metrics.WatermarkGauge; import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.OperatorChain; import org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask; @@ -289,6 +290,11 @@ public void emitWatermarkStatus(WatermarkStatus watermarkStatus) throws Exceptio public void emitLatencyMarker(LatencyMarker latencyMarker) throws Exception { input.processLatencyMarker(latencyMarker); } + + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) throws Exception { + input.processRecordAttributes(recordAttributes); + } } @SuppressWarnings({"unchecked", "rawtypes"}) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessorFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessorFactory.java index dbfc295ace0ede..c7463e4fded12d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessorFactory.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessorFactory.java @@ -40,6 +40,7 @@ import org.apache.flink.streaming.runtime.metrics.WatermarkGauge; import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.OperatorChain; import org.apache.flink.streaming.runtime.tasks.StreamTask.CanEmitBatchOfRecordsChecker; @@ -289,6 +290,15 @@ public void emitLatencyMarker(LatencyMarker latencyMarker) throws Exception { operator.processLatencyMarker2(latencyMarker); } } + + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) throws Exception { + if (inputIndex == 0) { + operator.processRecordAttributes1(recordAttributes); + } else { + operator.processRecordAttributes2(recordAttributes); + } + } } private static class FinishedOnRestoreWatermarkBypass { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/InternalRecordAttributes.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/InternalRecordAttributes.java new file mode 100644 index 00000000000000..3cd08d1cdef2d5 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/InternalRecordAttributes.java @@ -0,0 +1,67 @@ +/* + * 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.streaming.runtime.streamrecord; + +import org.apache.flink.annotation.Internal; + +import java.util.Objects; + +/** The RecordAttributes that contains extra information to be used internally. */ +@Internal +public class InternalRecordAttributes extends RecordAttributes { + private final long maxWatermarkDuringBacklog; + + public InternalRecordAttributes(boolean backlog, long maxWatermarkDuringBacklog) { + super(backlog); + this.maxWatermarkDuringBacklog = maxWatermarkDuringBacklog; + } + + public long getMaxWatermarkDuringBacklog() { + return maxWatermarkDuringBacklog; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + InternalRecordAttributes that = (InternalRecordAttributes) o; + return maxWatermarkDuringBacklog == that.maxWatermarkDuringBacklog; + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), maxWatermarkDuringBacklog); + } + + @Override + public String toString() { + return "InternalRecordAttributes{" + + "backlog=" + + isBacklog() + + ", watermarkDuringBacklog=" + + maxWatermarkDuringBacklog + + '}'; + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributes.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributes.java new file mode 100644 index 00000000000000..72007c1f0ec50e --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributes.java @@ -0,0 +1,66 @@ +/* + * 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.streaming.runtime.streamrecord; + +import org.apache.flink.annotation.Experimental; + +import java.util.Objects; + +/** + * A RecordAttributes element provides stream task with information that can be used to optimize the + * stream task's performance. + */ +@Experimental +public class RecordAttributes extends StreamElement { + private final boolean backlog; + + public RecordAttributes(boolean backlog) { + this.backlog = backlog; + } + + /** + * If it returns true, then the records received after this element are stale and an operator + * can optionally buffer records until isBacklog=false. This allows an operator to optimize + * throughput at the cost of processing latency. + */ + public boolean isBacklog() { + return backlog; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + RecordAttributes that = (RecordAttributes) o; + return backlog == that.backlog; + } + + @Override + public int hashCode() { + return Objects.hash(backlog); + } + + @Override + public String toString() { + return "RecordAttributes{" + "backlog=" + backlog + '}'; + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributesBuilder.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributesBuilder.java new file mode 100644 index 00000000000000..be1a3813491de2 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributesBuilder.java @@ -0,0 +1,77 @@ +/* + * 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.streaming.runtime.streamrecord; + +import org.apache.flink.annotation.Experimental; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.List; + +/** The builder class for {@link RecordAttributes}. */ +@Experimental +public class RecordAttributesBuilder { + private static final Logger LOG = LoggerFactory.getLogger(RecordAttributesBuilder.class); + private final List lastRecordAttributesOfInputs; + @Nullable private Boolean backlog = null; + + /** + * This constructor takes a list of the last recordAttributes received from each of the + * operator's inputs. When this list is not empty, it will be used to determine the default + * values for those attributes that have not been explicitly set by caller. + */ + public RecordAttributesBuilder(List lastRecordAttributesOfInputs) { + this.lastRecordAttributesOfInputs = lastRecordAttributesOfInputs; + } + + public RecordAttributesBuilder setBacklog(boolean backlog) { + this.backlog = backlog; + return this; + } + + /** + * If any operator attribute is null, we will log it at DEBUG level and determine a non-null + * default value as described below. + * + *

Default value for backlog: if any element in lastRecordAttributesOfInputs has + * backlog=true, use true. Otherwise, use false. + */ + public RecordAttributes build() { + if (backlog == null) { + final boolean defaultBacklog = getDefaultBacklog(); + LOG.debug( + "backlog is not set, set to {} from the last record attributes {}.", + defaultBacklog, + lastRecordAttributesOfInputs); + backlog = defaultBacklog; + } + return new RecordAttributes(backlog); + } + + private boolean getDefaultBacklog() { + for (RecordAttributes lastAttributes : lastRecordAttributesOfInputs) { + if (lastAttributes.isBacklog()) { + return true; + } + } + return false; + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElement.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElement.java index 3fbcf2e42dc353..c65b0b5bf7b1d1 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElement.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElement.java @@ -22,7 +22,7 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; -/** An element in a data stream. Can be a record or a Watermark. */ +/** An element in a data stream. Can be a record, a Watermark, or a RecordAttributes. */ @Internal public abstract class StreamElement { @@ -62,6 +62,15 @@ public final boolean isLatencyMarker() { return getClass() == LatencyMarker.class; } + /** + * Check whether this element is record attributes. + * + * @return True, if this element is record attributes, false otherwise. + */ + public final boolean isRecordAttributes() { + return getClass() == RecordAttributes.class; + } + /** * Casts this element into a StreamRecord. * @@ -103,4 +112,15 @@ public final WatermarkStatus asWatermarkStatus() { public final LatencyMarker asLatencyMarker() { return (LatencyMarker) this; } + + /** + * Casts this element into a RecordAttributes. + * + * @return This element as a RecordAttributes. + * @throws java.lang.ClassCastException Thrown, if this element is actually not a + * RecordAttributes. + */ + public final RecordAttributes asRecordAttributes() { + return (RecordAttributes) this; + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializer.java index b1052131769d17..4140c52a9ecfba 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializer.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializer.java @@ -50,6 +50,7 @@ public final class StreamElementSerializer extends TypeSerializer typeSerializer; @@ -146,6 +147,8 @@ public void copy(DataInputView source, DataOutputView target) throws IOException target.writeLong(source.readLong()); target.writeLong(source.readLong()); target.writeInt(source.readInt()); + } else if (tag == TAG_RECORD_ATTRIBUTES) { + target.writeBoolean(source.readBoolean()); } else { throw new IOException("Corrupt stream, found tag: " + tag); } @@ -175,6 +178,9 @@ public void serialize(StreamElement value, DataOutputView target) throws IOExcep target.writeLong(value.asLatencyMarker().getOperatorId().getLowerPart()); target.writeLong(value.asLatencyMarker().getOperatorId().getUpperPart()); target.writeInt(value.asLatencyMarker().getSubtaskIndex()); + } else if (value.isRecordAttributes()) { + target.write(TAG_RECORD_ATTRIBUTES); + target.writeBoolean(value.asRecordAttributes().isBacklog()); } else { throw new RuntimeException(); } @@ -197,6 +203,8 @@ public StreamElement deserialize(DataInputView source) throws IOException { source.readLong(), new OperatorID(source.readLong(), source.readLong()), source.readInt()); + } else if (tag == TAG_RECORD_ATTRIBUTES) { + return new RecordAttributes(source.readBoolean()); } else { throw new IOException("Corrupt stream, found tag: " + tag); } @@ -223,6 +231,8 @@ public StreamElement deserialize(StreamElement reuse, DataInputView source) thro source.readLong(), new OperatorID(source.readLong(), source.readLong()), source.readInt()); + } else if (tag == TAG_RECORD_ATTRIBUTES) { + return new RecordAttributes(source.readBoolean()); } else { throw new IOException("Corrupt stream, found tag: " + tag); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/BroadcastingOutputCollector.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/BroadcastingOutputCollector.java index 07f9a4d7677cbd..0f4613046ef9ca 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/BroadcastingOutputCollector.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/BroadcastingOutputCollector.java @@ -23,6 +23,7 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.metrics.WatermarkGauge; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import org.apache.flink.util.OutputTag; @@ -103,4 +104,11 @@ public void close() { output.close(); } } + + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) { + for (OutputWithChainingCheck> output : outputs) { + output.emitRecordAttributes(recordAttributes); + } + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/ChainingOutput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/ChainingOutput.java index 13c3bc6a9b64d6..1ffe6cfb14bc0b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/ChainingOutput.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/ChainingOutput.java @@ -26,6 +26,7 @@ import org.apache.flink.streaming.runtime.io.RecordProcessorUtils; import org.apache.flink.streaming.runtime.metrics.WatermarkGauge; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import org.apache.flink.util.OutputTag; @@ -154,4 +155,13 @@ public void emitWatermarkStatus(WatermarkStatus watermarkStatus) { } } } + + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) { + try { + input.processRecordAttributes(recordAttributes); + } catch (Exception e) { + throw new ExceptionInChainedOperatorException(e); + } + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java index bc3abb489ec984..ea1533b3bb0d28 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java @@ -29,6 +29,7 @@ import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.Input; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.sort.SortingBacklogDataInput; import org.apache.flink.streaming.api.operators.sort.SortingDataInput; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput.DataOutput; @@ -42,6 +43,7 @@ import org.apache.flink.streaming.runtime.io.checkpointing.InputProcessorUtil; import org.apache.flink.streaming.runtime.metrics.WatermarkGauge; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.watermarkstatus.StatusWatermarkValve; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; @@ -57,6 +59,7 @@ import java.util.Optional; import static org.apache.flink.streaming.api.graph.StreamConfig.requiresSorting; +import static org.apache.flink.streaming.api.graph.StreamConfig.requiresSortingDuringBacklog; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; @@ -112,6 +115,8 @@ public void init() throws Exception { !configuration.isCheckpointingEnabled(), "Checkpointing is not allowed with sorted inputs."); input = wrapWithSorted(input); + } else if (requiresSortingDuringBacklog(inputConfig)) { + input = wrapWithBacklogSorted(input); } getEnvironment() @@ -154,6 +159,26 @@ private StreamTaskInput wrapWithSorted(StreamTaskInput input) { getExecutionConfig()); } + private StreamTaskInput wrapWithBacklogSorted(StreamTaskInput input) { + ClassLoader userCodeClassLoader = getUserCodeClassLoader(); + return new SortingBacklogDataInput<>( + input, + configuration.getTypeSerializerIn(input.getInputIndex(), userCodeClassLoader), + configuration.getStateKeySerializer(userCodeClassLoader), + configuration.getStatePartitioner(input.getInputIndex(), userCodeClassLoader), + getEnvironment().getMemoryManager(), + getEnvironment().getIOManager(), + getExecutionConfig().isObjectReuseEnabled(), + configuration.getManagedMemoryFractionOperatorUseCaseOfSlot( + ManagedMemoryUseCase.OPERATOR, + getEnvironment().getTaskConfiguration(), + userCodeClassLoader), + getEnvironment().getTaskManagerInfo().getConfiguration(), + this, + getExecutionConfig(), + getCanEmitBatchOfRecords()); + } + @SuppressWarnings("unchecked") private CheckpointedInputGate createCheckpointedInputGate() { IndexedInputGate[] inputGates = getEnvironment().getAllInputGates(); @@ -252,5 +277,10 @@ public void emitWatermarkStatus(WatermarkStatus watermarkStatus) throws Exceptio public void emitLatencyMarker(LatencyMarker latencyMarker) throws Exception { operator.processLatencyMarker(latencyMarker); } + + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) throws Exception { + operator.processRecordAttributes(recordAttributes); + } } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTask.java index 70fa37fc1d0cad..c87ee6eead7c72 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTask.java @@ -39,6 +39,7 @@ import org.apache.flink.streaming.runtime.io.StreamTaskSourceInput; import org.apache.flink.streaming.runtime.metrics.WatermarkGauge; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import org.apache.flink.util.concurrent.FutureUtils; @@ -314,6 +315,11 @@ public void emitLatencyMarker(LatencyMarker latencyMarker) { output.emitLatencyMarker(latencyMarker); } + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) { + output.emitRecordAttributes(recordAttributes); + } + @Override public void emitWatermark(Watermark watermark) { long watermarkTimestamp = watermark.getTimestamp(); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/BacklogUtils.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/BacklogUtils.java new file mode 100644 index 00000000000000..f82d35bac20063 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/BacklogUtils.java @@ -0,0 +1,68 @@ +/* + * 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.streaming.runtime.translators; + +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ExecutionOptions; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.core.memory.ManagedMemoryUseCase; +import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.api.graph.StreamNode; +import org.apache.flink.streaming.api.graph.TransformationTranslator; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL_DURING_BACKLOG; + +/** A utility class for applying sorting inputs during backlog processing. */ +public class BacklogUtils { + public static boolean isCheckpointDisableDuringBacklog( + TransformationTranslator.Context context) { + return context.getGraphGeneratorConfig().get(CHECKPOINTING_INTERVAL_DURING_BACKLOG) != null + && context.getGraphGeneratorConfig() + .get(CHECKPOINTING_INTERVAL_DURING_BACKLOG) + .isZero(); + } + + public static void applyBacklogProcessingSettings( + Transformation transformation, + TransformationTranslator.Context context, + StreamNode node, + boolean isInternalSorted) { + if (!isInternalSorted) { + node.addInputRequirement(0, StreamConfig.InputRequirement.SORTED_DURING_BACKLOG); + } + Map operatorScopeUseCaseWeights = new HashMap<>(); + Integer operatorMemoryWeights = + transformation + .getManagedMemoryOperatorScopeUseCaseWeights() + .get(ManagedMemoryUseCase.OPERATOR); + operatorScopeUseCaseWeights.put( + ManagedMemoryUseCase.OPERATOR, + operatorMemoryWeights == null + ? deriveMemoryWeight(context.getGraphGeneratorConfig()) + : operatorMemoryWeights); + node.setManagedMemoryUseCaseWeights(operatorScopeUseCaseWeights, Collections.emptySet()); + } + + private static int deriveMemoryWeight(ReadableConfig configuration) { + return Math.max(1, configuration.get(ExecutionOptions.SORTED_INPUTS_MEMORY).getMebiBytes()); + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/MultiInputTransformationTranslator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/MultiInputTransformationTranslator.java index c61146995c4458..b6dbcb4b4d7a5b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/MultiInputTransformationTranslator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/MultiInputTransformationTranslator.java @@ -64,7 +64,8 @@ protected Collection translateForBatchInternal( IntStream.range(0, inputs.size()) .mapToObj( idx -> { - if (keySelectors.get(idx) != null) { + if (keySelectors.get(idx) != null + && !transformation.isInternalSorterSupported()) { return StreamConfig.InputRequirement.SORTED; } else { return StreamConfig.InputRequirement.PASS_THROUGH; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/OneInputTransformationTranslator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/OneInputTransformationTranslator.java index a80b7c0692296d..5f38578fa4d2a7 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/OneInputTransformationTranslator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/OneInputTransformationTranslator.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.api.graph.StreamNode; import org.apache.flink.streaming.api.graph.TransformationTranslator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -52,8 +53,12 @@ public Collection translateForBatchInternal( context); boolean isKeyed = keySelector != null; if (isKeyed) { + final StreamConfig.InputRequirement inputRequirement = + transformation.isInternalSorterSupported() + ? StreamConfig.InputRequirement.PASS_THROUGH + : StreamConfig.InputRequirement.SORTED; BatchExecutionUtils.applyBatchExecutionSettings( - transformation.getId(), context, StreamConfig.InputRequirement.SORTED); + transformation.getId(), context, inputRequirement); } return ids; @@ -62,12 +67,20 @@ public Collection translateForBatchInternal( @Override public Collection translateForStreamingInternal( final OneInputTransformation transformation, final Context context) { - return translateInternal( - transformation, - transformation.getOperatorFactory(), - transformation.getInputType(), - transformation.getStateKeySelector(), - transformation.getStateKeyType(), - context); + final KeySelector keySelector = transformation.getStateKeySelector(); + final Collection ids = + translateInternal( + transformation, + transformation.getOperatorFactory(), + transformation.getInputType(), + keySelector, + transformation.getStateKeyType(), + context); + final StreamNode node = context.getStreamGraph().getStreamNode(transformation.getId()); + if (keySelector != null && BacklogUtils.isCheckpointDisableDuringBacklog(context)) { + BacklogUtils.applyBacklogProcessingSettings( + transformation, context, node, transformation.isInternalSorterSupported()); + } + return ids; } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/ReduceTransformationTranslator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/ReduceTransformationTranslator.java index 63bf563e5500a5..30910ac02df0d7 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/ReduceTransformationTranslator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/ReduceTransformationTranslator.java @@ -18,7 +18,9 @@ package org.apache.flink.streaming.runtime.translators; +import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.api.graph.StreamNode; import org.apache.flink.streaming.api.graph.TransformationTranslator; import org.apache.flink.streaming.api.operators.BatchGroupedReduceOperator; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; @@ -54,8 +56,12 @@ public Collection translateForBatchInternal( transformation.getKeySelector(), transformation.getKeyTypeInfo(), context); + final StreamConfig.InputRequirement inputRequirement = + transformation.isInternalSorterSupported() + ? StreamConfig.InputRequirement.PASS_THROUGH + : StreamConfig.InputRequirement.SORTED; BatchExecutionUtils.applyBatchExecutionSettings( - transformation.getId(), context, StreamConfig.InputRequirement.SORTED); + transformation.getId(), context, inputRequirement); return ids; } @@ -72,12 +78,21 @@ public Collection translateForStreamingInternal( SimpleOperatorFactory operatorFactory = SimpleOperatorFactory.of(groupedReduce); operatorFactory.setChainingStrategy(transformation.getChainingStrategy()); - return translateInternal( - transformation, - operatorFactory, - transformation.getInputType(), - transformation.getKeySelector(), - transformation.getKeyTypeInfo(), - context); + + final KeySelector keySelector = transformation.getKeySelector(); + final Collection ids = + translateInternal( + transformation, + operatorFactory, + transformation.getInputType(), + keySelector, + transformation.getKeyTypeInfo(), + context); + final StreamNode node = context.getStreamGraph().getStreamNode(transformation.getId()); + if (keySelector != null && BacklogUtils.isCheckpointDisableDuringBacklog(context)) { + BacklogUtils.applyBacklogProcessingSettings( + transformation, context, node, transformation.isInternalSorterSupported()); + } + return ids; } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/TwoInputTransformationTranslator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/TwoInputTransformationTranslator.java index fde3a1aae760ad..ceb9f393648e97 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/TwoInputTransformationTranslator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/TwoInputTransformationTranslator.java @@ -46,11 +46,13 @@ protected Collection translateForBatchInternal( StreamConfig.InputRequirement input1Requirement = transformation.getStateKeySelector1() != null + && !transformation.isInternalSorterSupported() ? StreamConfig.InputRequirement.SORTED : StreamConfig.InputRequirement.PASS_THROUGH; StreamConfig.InputRequirement input2Requirement = transformation.getStateKeySelector2() != null + && !transformation.isInternalSorterSupported() ? StreamConfig.InputRequirement.SORTED : StreamConfig.InputRequirement.PASS_THROUGH; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java index 3d4125bac394a8..6833271f728bde 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java @@ -99,6 +99,7 @@ public class AbstractUdfStreamOperatorLifecycleTest { + "finish[], " + "getCurrentKey[], " + "getMetricGroup[], " + + "getOperatorAttributes[], " + "getOperatorID[], " + "initializeState[interface org.apache.flink.streaming.api.operators.StreamTaskStateInitializer], " + "notifyCheckpointAborted[long], " diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/BacklogTimeServiceTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/BacklogTimeServiceTest.java new file mode 100644 index 00000000000000..863791666e0e90 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/BacklogTimeServiceTest.java @@ -0,0 +1,75 @@ +/* + * 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.streaming.api.operators; + +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.heap.HeapPriorityQueueSetFactory; +import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; + +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flink.streaming.api.operators.TimeServiceTestUtils.createBacklogTimerService; +import static org.apache.flink.streaming.api.operators.TimeServiceTestUtils.createTimerQueue; +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link BacklogTimeService}. */ +public class BacklogTimeServiceTest { + + @Test + public void testTriggerEventTimeTimer() throws Exception { + List timers = new ArrayList<>(); + TestProcessingTimeService processingTimeService = new TestProcessingTimeService(); + KeyGroupRange testKeyGroupRange = new KeyGroupRange(0, 1); + final HeapPriorityQueueSetFactory priorityQueueSetFactory = + new HeapPriorityQueueSetFactory(testKeyGroupRange, 1, 128); + + final TimerSerializer timerSerializer = + new TimerSerializer<>(IntSerializer.INSTANCE, StringSerializer.INSTANCE); + final BacklogTimeService timeService = + createBacklogTimerService( + processingTimeService, + TestTrigger.eventTimeTrigger((timer) -> timers.add(timer.getTimestamp())), + createTimerQueue( + "eventTimerQueue", timerSerializer, priorityQueueSetFactory)); + + timeService.setMaxWatermarkDuringBacklog(2); + timeService.setCurrentKey(1); + timeService.registerEventTimeTimer("a", 0); + timeService.registerEventTimeTimer("a", 2); + timeService.registerEventTimeTimer("a", 1); + timeService.registerEventTimeTimer("a", 3); + assertThat(timers).isEmpty(); + timeService.setCurrentKey(2); + assertThat(timers).containsExactly(0L, 1L, 2L); + timers.clear(); + + timeService.registerEventTimeTimer("a", 2); + timeService.registerEventTimeTimer("a", 1); + timeService.registerEventTimeTimer("a", 3); + assertThat(timers).isEmpty(); + timeService.setCurrentKey(null); + assertThat(timers).containsExactly(1L, 2L); + + assertThat(timeService.currentWatermark()).isEqualTo(2); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/InternalBacklogAwareTimerServiceImplTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/InternalBacklogAwareTimerServiceImplTest.java new file mode 100644 index 00000000000000..1f0961b3c9ab35 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/InternalBacklogAwareTimerServiceImplTest.java @@ -0,0 +1,111 @@ +/* + * 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.streaming.api.operators; + +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue; +import org.apache.flink.runtime.state.heap.HeapPriorityQueueSetFactory; +import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; + +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flink.streaming.api.operators.TimeServiceTestUtils.createBacklogTimerService; +import static org.apache.flink.streaming.api.operators.TimeServiceTestUtils.createInternalTimerService; +import static org.apache.flink.streaming.api.operators.TimeServiceTestUtils.createTimerQueue; +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link InternalBacklogAwareTimerServiceImpl}. */ +public class InternalBacklogAwareTimerServiceImplTest { + + @Test + public void testTriggerEventTimeTimerWithBacklog() throws Exception { + KeyGroupRange testKeyGroupRange = new KeyGroupRange(0, 1); + List> timers = new ArrayList<>(); + TestProcessingTimeService processingTimeService = new TestProcessingTimeService(); + final HeapPriorityQueueSetFactory priorityQueueSetFactory = + new HeapPriorityQueueSetFactory(testKeyGroupRange, 1, 128); + TimerSerializer timerSerializer = + new TimerSerializer<>(IntSerializer.INSTANCE, StringSerializer.INSTANCE); + final TestTrigger triggerable = + TestTrigger.eventTimeTrigger( + (timer) -> timers.add(Tuple2.of(timer.getKey(), timer.getTimestamp()))); + final KeyGroupedInternalPriorityQueue> + eventTimersQueue = + createTimerQueue( + "eventTimersQueue", timerSerializer, priorityQueueSetFactory); + final KeyGroupedInternalPriorityQueue> + processingTimerQueue = + createTimerQueue( + "processingTimerQueue", timerSerializer, priorityQueueSetFactory); + final BacklogTimeService backlogTimeService = + createBacklogTimerService(processingTimeService, triggerable, eventTimersQueue); + final TestKeyContext keyContext = new TestKeyContext(); + final InternalTimerServiceImpl internalTimerService = + createInternalTimerService( + testKeyGroupRange, + keyContext, + processingTimeService, + processingTimerQueue, + eventTimersQueue); + internalTimerService.startTimerService( + IntSerializer.INSTANCE, StringSerializer.INSTANCE, triggerable); + + final InternalBacklogAwareTimerServiceImpl timerService = + new InternalBacklogAwareTimerServiceImpl<>( + internalTimerService, backlogTimeService); + + keyContext.setCurrentKey(1); + timerService.registerEventTimeTimer("a", 2); + timerService.registerEventTimeTimer("a", 1); + timerService.registerEventTimeTimer("a", 3); + keyContext.setCurrentKey(2); + timerService.registerEventTimeTimer("a", 3); + timerService.registerEventTimeTimer("a", 1); + timerService.advanceWatermark(2); + assertThat(timers).containsExactly(Tuple2.of(1, 1L), Tuple2.of(2, 1L), Tuple2.of(1, 2L)); + timers.clear(); + + // switch to backlog processing + timerService.setMaxWatermarkDuringBacklog(5); + timerService.setBacklog(true); + timerService.setCurrentKey(1); + timerService.registerEventTimeTimer("a", 5); + timerService.registerEventTimeTimer("a", 4); + timerService.setCurrentKey(2); + timerService.registerEventTimeTimer("a", 6); + timerService.setCurrentKey(null); + assertThat(timers) + .containsExactly( + Tuple2.of(1, 3L), Tuple2.of(1, 4L), Tuple2.of(1, 5L), Tuple2.of(2, 3L)); + timers.clear(); + + // switch to non backlog processing + timerService.setBacklog(false); + assertThat(timerService.currentWatermark()).isEqualTo(5); + keyContext.setCurrentKey(1); + timerService.registerEventTimeTimer("a", 6); + timerService.advanceWatermark(6); + assertThat(timers).containsExactly(Tuple2.of(2, 6L), Tuple2.of(1, 6L)); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImplTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImplTest.java index 915b2732982653..889eae53654bbd 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImplTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImplTest.java @@ -18,7 +18,6 @@ package org.apache.flink.streaming.api.operators; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.IntSerializer; import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.java.tuple.Tuple3; @@ -30,7 +29,6 @@ import org.apache.flink.runtime.state.PriorityQueueSetFactory; import org.apache.flink.runtime.state.heap.HeapPriorityQueueSetFactory; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; -import org.apache.flink.streaming.runtime.tasks.StreamTaskCancellationContext; import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; import org.junit.Assert; @@ -51,6 +49,7 @@ import java.util.Random; import java.util.Set; +import static org.apache.flink.streaming.api.operators.TimeServiceTestUtils.createInternalTimerService; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; @@ -963,21 +962,6 @@ private void testSnapshotAndRebalancingRestore(int snapshotVersion) throws Excep assertEquals(0, timerService2.numEventTimeTimers()); } - private static class TestKeyContext implements KeyContext { - - private Object key; - - @Override - public void setCurrentKey(Object key) { - this.key = key; - } - - @Override - public Object getCurrentKey() { - return key; - } - } - private static int getKeyInKeyGroup(int keyGroup, int maxParallelism) { Random rand = new Random(System.currentTimeMillis()); int result = rand.nextInt(); @@ -1084,27 +1068,6 @@ public static Collection keyRanges() { }); } - private static InternalTimerServiceImpl createInternalTimerService( - KeyGroupRange keyGroupsList, - KeyContext keyContext, - ProcessingTimeService processingTimeService, - TypeSerializer keySerializer, - TypeSerializer namespaceSerializer, - PriorityQueueSetFactory priorityQueueSetFactory) { - - TimerSerializer timerSerializer = - new TimerSerializer<>(keySerializer, namespaceSerializer); - - return new InternalTimerServiceImpl<>( - keyGroupsList, - keyContext, - processingTimeService, - createTimerQueue( - "__test_processing_timers", timerSerializer, priorityQueueSetFactory), - createTimerQueue("__test_event_timers", timerSerializer, priorityQueueSetFactory), - StreamTaskCancellationContext.alwaysRunning()); - } - private static KeyGroupedInternalPriorityQueue> createTimerQueue( String name, diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TestKeyContext.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TestKeyContext.java new file mode 100644 index 00000000000000..2dfb0af4779101 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TestKeyContext.java @@ -0,0 +1,34 @@ +/* + * 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.streaming.api.operators; + +/** KeyContext for test. */ +class TestKeyContext implements KeyContext { + + private Object key; + + @Override + public void setCurrentKey(Object key) { + this.key = key; + } + + @Override + public Object getCurrentKey() { + return key; + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TestTrigger.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TestTrigger.java new file mode 100644 index 00000000000000..3f262180f33b86 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TestTrigger.java @@ -0,0 +1,64 @@ +/* + * 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.streaming.api.operators; + +import org.assertj.core.api.Assertions; + +import java.util.function.Consumer; + +/** + * TestTrigger is a Triggerable for test. The test trigger take an event time handler or processing + * time handler, which will be invoked when the trigger is triggered on event time or processing + * time. + */ +public class TestTrigger implements Triggerable { + + private final Consumer> eventTimeHandler; + private final Consumer> processingTimeHandler; + + public static TestTrigger eventTimeTrigger( + Consumer> eventTimeHandler) { + return new TestTrigger<>( + eventTimeHandler, + timer -> Assertions.fail("We did not expect processing timer to be triggered.")); + } + + public static TestTrigger processingTimeTrigger( + Consumer> processingTimeHandler) { + return new TestTrigger<>( + timer -> Assertions.fail("We did not expect event timer to be triggered."), + processingTimeHandler); + } + + private TestTrigger( + Consumer> eventTimeHandler, + Consumer> processingTimeHandler) { + this.eventTimeHandler = eventTimeHandler; + this.processingTimeHandler = processingTimeHandler; + } + + @Override + public void onEventTime(InternalTimer timer) throws Exception { + this.eventTimeHandler.accept(timer); + } + + @Override + public void onProcessingTime(InternalTimer timer) throws Exception { + this.processingTimeHandler.accept(timer); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TimeServiceTestUtils.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TimeServiceTestUtils.java new file mode 100644 index 00000000000000..2808b75f52d920 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TimeServiceTestUtils.java @@ -0,0 +1,81 @@ +/* + * 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.streaming.api.operators; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue; +import org.apache.flink.runtime.state.PriorityQueueSetFactory; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.StreamTaskCancellationContext; + +/** Util methods for TimeService tests. */ +public class TimeServiceTestUtils { + + public static + KeyGroupedInternalPriorityQueue> createTimerQueue( + String name, + TimerSerializer timerSerializer, + PriorityQueueSetFactory priorityQueueSetFactory) { + return priorityQueueSetFactory.create(name, timerSerializer); + } + + public static InternalTimerServiceImpl createInternalTimerService( + KeyGroupRange keyGroupsList, + KeyContext keyContext, + ProcessingTimeService processingTimeService, + TypeSerializer keySerializer, + TypeSerializer namespaceSerializer, + PriorityQueueSetFactory priorityQueueSetFactory) { + + TimerSerializer timerSerializer = + new TimerSerializer<>(keySerializer, namespaceSerializer); + + return createInternalTimerService( + keyGroupsList, + keyContext, + processingTimeService, + createTimerQueue( + "__test_processing_timers", timerSerializer, priorityQueueSetFactory), + createTimerQueue("__test_event_timers", timerSerializer, priorityQueueSetFactory)); + } + + public static InternalTimerServiceImpl createInternalTimerService( + KeyGroupRange keyGroupsList, + KeyContext keyContext, + ProcessingTimeService processingTimeService, + KeyGroupedInternalPriorityQueue> processingTimeTimersQueue, + KeyGroupedInternalPriorityQueue> eventTimeTimersQueue) { + + return new InternalTimerServiceImpl<>( + keyGroupsList, + keyContext, + processingTimeService, + processingTimeTimersQueue, + eventTimeTimersQueue, + StreamTaskCancellationContext.alwaysRunning()); + } + + public static BacklogTimeService createBacklogTimerService( + ProcessingTimeService processingTimeService, + Triggerable triggerable, + KeyGroupedInternalPriorityQueue> eventTimeTimersQueue) { + + return new BacklogTimeService<>(processingTimeService, triggerable, eventTimeTimersQueue); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/sort/CollectingDataOutput.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/sort/CollectingDataOutput.java index 1c5a9ae4fcfa21..ca8e061cf07159 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/sort/CollectingDataOutput.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/sort/CollectingDataOutput.java @@ -21,6 +21,7 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; @@ -53,4 +54,9 @@ public void emitRecord(StreamRecord streamRecord) throws Exception { public void emitLatencyMarker(LatencyMarker latencyMarker) throws Exception { events.add(latencyMarker); } + + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) throws Exception { + events.add(recordAttributes); + } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/sort/CollectionDataInput.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/sort/CollectionDataInput.java index 2926c3acd05aaa..bffe244c40706d 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/sort/CollectionDataInput.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/sort/CollectionDataInput.java @@ -53,6 +53,8 @@ public DataInputStatus emitNext(DataOutput output) throws Exception { output.emitRecord(streamElement.asRecord()); } else if (streamElement instanceof Watermark) { output.emitWatermark(streamElement.asWatermark()); + } else if (streamElement.isRecordAttributes()) { + output.emitRecordAttributes(streamElement.asRecordAttributes()); } else { throw new IllegalStateException("Unsupported element type: " + streamElement); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/sort/LargeSortingDataInputITCase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/sort/LargeSortingDataInputITCase.java index 21aa798b12f74f..1d6a76547dc5ce 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/sort/LargeSortingDataInputITCase.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/sort/LargeSortingDataInputITCase.java @@ -42,6 +42,7 @@ import org.apache.flink.streaming.runtime.io.StreamOneInputProcessor; import org.apache.flink.streaming.runtime.io.StreamTaskInput; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; @@ -219,6 +220,9 @@ public void emitWatermarkStatus(WatermarkStatus watermarkStatus) throws Exceptio @Override public void emitLatencyMarker(LatencyMarker latencyMarker) throws Exception {} + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) throws Exception {} + public int getSeenRecords() { return seenRecords; } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/sort/SortingBacklogDataInputTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/sort/SortingBacklogDataInputTest.java new file mode 100644 index 00000000000000..6afb9232dccf95 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/sort/SortingBacklogDataInputTest.java @@ -0,0 +1,235 @@ +/* + * 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.streaming.api.operators.sort; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.operators.testutils.DummyInvokable; +import org.apache.flink.runtime.operators.testutils.MockEnvironment; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.io.DataInputStatus; +import org.apache.flink.streaming.runtime.streamrecord.InternalRecordAttributes; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributesBuilder; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; + +/** Tests for {@link SortingBacklogDataInput}. */ +public class SortingBacklogDataInputTest { + @Test + public void sortingDuringBacklog() throws Exception { + CollectingDataOutput collectingDataOutput = new CollectingDataOutput<>(); + CollectionDataInput input = + new CollectionDataInput<>( + Arrays.asList( + new StreamRecord<>(2, 0), + new StreamRecord<>(1, 0), + new RecordAttributesBuilder(Collections.emptyList()) + .setBacklog(true) + .build(), + new StreamRecord<>(1, 3), + new StreamRecord<>(1, 1), + new StreamRecord<>(2, 1), + new StreamRecord<>(2, 3), + new StreamRecord<>(1, 2), + new StreamRecord<>(2, 2), + new RecordAttributesBuilder(Collections.emptyList()) + .setBacklog(false) + .build(), + new StreamRecord<>(1, 4), + new StreamRecord<>(2, 4), + new StreamRecord<>(1, 5))); + MockEnvironment environment = MockEnvironment.builder().build(); + SortingBacklogDataInput sortingDataInput = + new SortingBacklogDataInput<>( + input, + new IntSerializer(), + new IntSerializer(), + (KeySelector) value -> value, + environment.getMemoryManager(), + environment.getIOManager(), + true, + 1.0, + new Configuration(), + new DummyInvokable(), + new ExecutionConfig(), + () -> true); + + DataInputStatus inputStatus; + do { + inputStatus = sortingDataInput.emitNext(collectingDataOutput); + } while (inputStatus != DataInputStatus.END_OF_INPUT); + + org.assertj.core.api.Assertions.assertThat(collectingDataOutput.events) + .containsExactly( + new StreamRecord<>(2, 0), + new StreamRecord<>(1, 0), + new RecordAttributesBuilder(Collections.emptyList()) + .setBacklog(true) + .build(), + new InternalRecordAttributes(true, Long.MIN_VALUE), + new StreamRecord<>(1, 1), + new StreamRecord<>(1, 2), + new StreamRecord<>(1, 3), + new StreamRecord<>(2, 1), + new StreamRecord<>(2, 2), + new StreamRecord<>(2, 3), + new RecordAttributesBuilder(Collections.emptyList()) + .setBacklog(false) + .build(), + new StreamRecord<>(1, 4), + new StreamRecord<>(2, 4), + new StreamRecord<>(1, 5)); + } + + @Test + public void watermarkPropagation() throws Exception { + CollectingDataOutput collectingDataOutput = new CollectingDataOutput<>(); + CollectionDataInput input = + new CollectionDataInput<>( + Arrays.asList( + new StreamRecord<>(1, 3), + new Watermark(1), + new RecordAttributesBuilder(Collections.emptyList()) + .setBacklog(true) + .build(), + new StreamRecord<>(1, 1), + new Watermark(2), + new StreamRecord<>(2, 1), + new Watermark(3), + new StreamRecord<>(2, 3), + new Watermark(4), + new StreamRecord<>(1, 2), + new Watermark(5), + new StreamRecord<>(2, 2), + new Watermark(6), + new RecordAttributesBuilder(Collections.emptyList()) + .setBacklog(false) + .build())); + MockEnvironment environment = MockEnvironment.builder().build(); + SortingBacklogDataInput sortingDataInput = + new SortingBacklogDataInput<>( + input, + new IntSerializer(), + new IntSerializer(), + (KeySelector) value -> value, + environment.getMemoryManager(), + environment.getIOManager(), + true, + 1.0, + new Configuration(), + new DummyInvokable(), + new ExecutionConfig(), + () -> true); + + DataInputStatus inputStatus; + do { + inputStatus = sortingDataInput.emitNext(collectingDataOutput); + } while (inputStatus != DataInputStatus.END_OF_INPUT); + + org.assertj.core.api.Assertions.assertThat(collectingDataOutput.events) + .containsExactly( + new StreamRecord<>(1, 3), + new Watermark(1), + new RecordAttributesBuilder(Collections.emptyList()) + .setBacklog(true) + .build(), + new InternalRecordAttributes(true, 6L), + new StreamRecord<>(1, 1), + new StreamRecord<>(1, 2), + new StreamRecord<>(2, 1), + new StreamRecord<>(2, 2), + new StreamRecord<>(2, 3), + new RecordAttributesBuilder(Collections.emptyList()) + .setBacklog(false) + .build(), + new Watermark(6)); + } + + @Test + public void simpleVariableLengthKeySorting() throws Exception { + CollectingDataOutput collectingDataOutput = new CollectingDataOutput<>(); + CollectionDataInput input = + new CollectionDataInput<>( + Arrays.asList( + new StreamRecord<>(2, 0), + new StreamRecord<>(1, 0), + new RecordAttributesBuilder(Collections.emptyList()) + .setBacklog(true) + .build(), + new StreamRecord<>(1, 3), + new StreamRecord<>(1, 1), + new StreamRecord<>(2, 1), + new StreamRecord<>(2, 3), + new StreamRecord<>(1, 2), + new StreamRecord<>(2, 2), + new RecordAttributesBuilder(Collections.emptyList()) + .setBacklog(false) + .build(), + new StreamRecord<>(1, 4), + new StreamRecord<>(2, 4), + new StreamRecord<>(1, 5))); + MockEnvironment environment = MockEnvironment.builder().build(); + SortingBacklogDataInput sortingDataInput = + new SortingBacklogDataInput<>( + input, + new IntSerializer(), + new StringSerializer(), + (KeySelector) value -> "" + value, + environment.getMemoryManager(), + environment.getIOManager(), + true, + 1.0, + new Configuration(), + new DummyInvokable(), + new ExecutionConfig(), + () -> true); + + DataInputStatus inputStatus; + do { + inputStatus = sortingDataInput.emitNext(collectingDataOutput); + } while (inputStatus != DataInputStatus.END_OF_INPUT); + + org.assertj.core.api.Assertions.assertThat(collectingDataOutput.events) + .containsExactly( + new StreamRecord<>(2, 0), + new StreamRecord<>(1, 0), + new RecordAttributesBuilder(Collections.emptyList()) + .setBacklog(true) + .build(), + new InternalRecordAttributes(true, Long.MIN_VALUE), + new StreamRecord<>(1, 1), + new StreamRecord<>(1, 2), + new StreamRecord<>(1, 3), + new StreamRecord<>(2, 1), + new StreamRecord<>(2, 2), + new StreamRecord<>(2, 3), + new RecordAttributesBuilder(Collections.emptyList()) + .setBacklog(false) + .build(), + new StreamRecord<>(1, 4), + new StreamRecord<>(2, 4), + new StreamRecord<>(1, 5)); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionInternalTimeServiceTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionInternalTimeServiceTest.java index 35932523ba60b7..7470fd18199e9d 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionInternalTimeServiceTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionInternalTimeServiceTest.java @@ -34,6 +34,7 @@ import org.apache.flink.streaming.api.operators.InternalTimer; import org.apache.flink.streaming.api.operators.InternalTimerService; import org.apache.flink.streaming.api.operators.KeyContext; +import org.apache.flink.streaming.api.operators.TestTrigger; import org.apache.flink.streaming.api.operators.Triggerable; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.tasks.StreamTaskCancellationContext; @@ -50,7 +51,6 @@ import java.util.Collections; import java.util.List; import java.util.function.BiConsumer; -import java.util.function.Consumer; import static org.hamcrest.CoreMatchers.equalTo; import static org.junit.Assert.assertThat; @@ -103,8 +103,7 @@ public void testForEachEventTimeTimerUnsupported() { BatchExecutionInternalTimeService timeService = new BatchExecutionInternalTimeService<>( - new TestProcessingTimeService(), - LambdaTrigger.eventTimeTrigger(timer -> {})); + new TestProcessingTimeService(), TestTrigger.eventTimeTrigger(timer -> {})); timeService.forEachEventTimeTimer( (o, aLong) -> fail("The forEachEventTimeTimer() should not be supported")); @@ -118,8 +117,7 @@ public void testForEachProcessingTimeTimerUnsupported() { BatchExecutionInternalTimeService timeService = new BatchExecutionInternalTimeService<>( - new TestProcessingTimeService(), - LambdaTrigger.eventTimeTrigger(timer -> {})); + new TestProcessingTimeService(), TestTrigger.eventTimeTrigger(timer -> {})); timeService.forEachEventTimeTimer( (o, aLong) -> fail("The forEachProcessingTimeTimer() should not be supported")); @@ -145,7 +143,7 @@ public void testFiringEventTimeTimers() throws Exception { "test", KEY_SERIALIZER, new VoidNamespaceSerializer(), - LambdaTrigger.eventTimeTrigger(timer -> timers.add(timer.getTimestamp()))); + TestTrigger.eventTimeTrigger(timer -> timers.add(timer.getTimestamp()))); keyedStatedBackend.setCurrentKey(1); timerService.registerEventTimeTimer(VoidNamespace.INSTANCE, 123); @@ -181,7 +179,7 @@ public void testSettingSameKeyDoesNotFireTimers() { "test", KEY_SERIALIZER, new VoidNamespaceSerializer(), - LambdaTrigger.eventTimeTrigger(timer -> timers.add(timer.getTimestamp()))); + TestTrigger.eventTimeTrigger(timer -> timers.add(timer.getTimestamp()))); keyedStatedBackend.setCurrentKey(1); timerService.registerEventTimeTimer(VoidNamespace.INSTANCE, 123); @@ -257,7 +255,7 @@ public void testProcessingTimeTimers() { "test", KEY_SERIALIZER, new VoidNamespaceSerializer(), - LambdaTrigger.processingTimeTrigger( + TestTrigger.processingTimeTrigger( timer -> timers.add(timer.getTimestamp()))); keyedStatedBackend.setCurrentKey(1); @@ -396,43 +394,6 @@ public void onProcessingTime(InternalTimer timer) throws Exception { } } - private static class LambdaTrigger implements Triggerable { - - private final Consumer> eventTimeHandler; - private final Consumer> processingTimeHandler; - - public static LambdaTrigger eventTimeTrigger( - Consumer> eventTimeHandler) { - return new LambdaTrigger<>( - eventTimeHandler, - timer -> Assert.fail("We did not expect processing timer to be triggered.")); - } - - public static LambdaTrigger processingTimeTrigger( - Consumer> processingTimeHandler) { - return new LambdaTrigger<>( - timer -> Assert.fail("We did not expect event timer to be triggered."), - processingTimeHandler); - } - - private LambdaTrigger( - Consumer> eventTimeHandler, - Consumer> processingTimeHandler) { - this.eventTimeHandler = eventTimeHandler; - this.processingTimeHandler = processingTimeHandler; - } - - @Override - public void onEventTime(InternalTimer timer) throws Exception { - this.eventTimeHandler.accept(timer); - } - - @Override - public void onProcessingTime(InternalTimer timer) throws Exception { - this.processingTimeHandler.accept(timer); - } - } - private static class DummyKeyContext implements KeyContext { @Override public void setCurrentKey(Object key) {} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/source/CollectingDataOutput.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/source/CollectingDataOutput.java index 6a6c535b891011..51cf3eb24019bd 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/source/CollectingDataOutput.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/source/CollectingDataOutput.java @@ -21,6 +21,7 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; @@ -56,6 +57,11 @@ public void emitLatencyMarker(LatencyMarker latencyMarker) throws Exception { events.add(latencyMarker); } + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) throws Exception { + events.add(recordAttributes); + } + public List getEvents() { return events; } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/RecordAttributesValveTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/RecordAttributesValveTest.java new file mode 100644 index 00000000000000..8836d79a489d01 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/RecordAttributesValveTest.java @@ -0,0 +1,58 @@ +/* + * 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.streaming.runtime.io; + +import org.apache.flink.streaming.api.operators.source.CollectingDataOutput; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributesBuilder; + +import org.junit.jupiter.api.Test; + +import java.util.Collections; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link RecordAttributesValve}. */ +public class RecordAttributesValveTest { + + @Test + public void testRecordAttributesValve() throws Exception { + final RecordAttributesValve valve = new RecordAttributesValve(3); + CollectingDataOutput collectingDataOutput = new CollectingDataOutput<>(); + final RecordAttributes backlogRecordAttribute = + new RecordAttributesBuilder(Collections.emptyList()).setBacklog(true).build(); + final RecordAttributes nonBacklogRecordAttribute = + new RecordAttributesBuilder(Collections.emptyList()).setBacklog(false).build(); + + // Switch from null to backlog + valve.inputRecordAttributes(backlogRecordAttribute, 0, collectingDataOutput); + valve.inputRecordAttributes(backlogRecordAttribute, 1, collectingDataOutput); + valve.inputRecordAttributes(backlogRecordAttribute, 2, collectingDataOutput); + + // Switch from backlog to non-backlog + valve.inputRecordAttributes(nonBacklogRecordAttribute, 0, collectingDataOutput); + valve.inputRecordAttributes(nonBacklogRecordAttribute, 1, collectingDataOutput); + + // Switch from non-backlog to backlog should be ignored + valve.inputRecordAttributes(backlogRecordAttribute, 0, collectingDataOutput); + valve.inputRecordAttributes(backlogRecordAttribute, 1, collectingDataOutput); + + assertThat(collectingDataOutput.getEvents()) + .containsExactly(backlogRecordAttribute, nonBacklogRecordAttribute); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java index 7338469171bb28..481bdaff082315 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java @@ -49,6 +49,7 @@ import org.apache.flink.streaming.runtime.io.checkpointing.SingleCheckpointBarrierHandler; import org.apache.flink.streaming.runtime.io.checkpointing.UpstreamRecoveryTracker; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamElement; import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -399,6 +400,9 @@ public void emitWatermarkStatus(WatermarkStatus watermarkStatus) {} @Override public void emitLatencyMarker(LatencyMarker latencyMarker) {} + + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) {} } private static class VerifyRecordsDataOutput extends NoOpDataOutput { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializerTest.java index dc4413288560a5..a328f5e9d03ae5 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializerTest.java @@ -29,6 +29,7 @@ import org.junit.Test; import java.io.IOException; +import java.util.Collections; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -92,6 +93,10 @@ public void testSerialization() throws Exception { LatencyMarker latencyMarker = new LatencyMarker(System.currentTimeMillis(), new OperatorID(-1, -1), 1); assertEquals(latencyMarker, serializeAndDeserialize(latencyMarker, serializer)); + + RecordAttributes recordAttributes = + new RecordAttributesBuilder(Collections.emptyList()).setBacklog(true).build(); + assertEquals(recordAttributes, serializeAndDeserialize(recordAttributes, serializer)); } @SuppressWarnings("unchecked") diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/watermarkstatus/StatusWatermarkValveTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/watermarkstatus/StatusWatermarkValveTest.java index 0db7b04e6c47f3..52bbc518541f16 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/watermarkstatus/StatusWatermarkValveTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/watermarkstatus/StatusWatermarkValveTest.java @@ -21,6 +21,7 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamElement; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -453,6 +454,11 @@ public void emitLatencyMarker(LatencyMarker latencyMarker) { throw new UnsupportedOperationException(); } + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) throws Exception { + throw new UnsupportedOperationException(); + } + public StreamElement popLastSeenOutput() { return allOutputs.poll(); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/SourceOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/SourceOperatorTestHarness.java index 2dbfa280411f37..0509ab0bc748d8 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/SourceOperatorTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/SourceOperatorTestHarness.java @@ -25,6 +25,7 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput.DataOutput; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; @@ -79,5 +80,10 @@ public void emitWatermarkStatus(WatermarkStatus watermarkStatus) { public void emitLatencyMarker(LatencyMarker latencyMarker) { output.emitLatencyMarker(latencyMarker); } + + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) throws Exception { + output.emitRecordAttributes(recordAttributes); + } } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/TimeTestUtil.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/TimeTestUtil.scala index b1ad73b7e2178b..8f22a3b494d6c5 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/TimeTestUtil.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/TimeTestUtil.scala @@ -25,7 +25,7 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext import org.apache.flink.streaming.api.operators.{AbstractStreamOperator, OneInputStreamOperator} import org.apache.flink.streaming.api.watermark.Watermark -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord +import org.apache.flink.streaming.runtime.streamrecord.{RecordAttributes, StreamRecord} import org.apache.flink.table.planner.JLong object TimeTestUtil { @@ -98,6 +98,8 @@ object TimeTestUtil { } } + override def processRecordAttributes(recordAttributes: RecordAttributes): Unit = + super.processRecordAttributes(recordAttributes) } } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/multipleinput/output/BroadcastingOutput.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/multipleinput/output/BroadcastingOutput.java index 6d158dda1ff3f1..166dfd16e06c28 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/multipleinput/output/BroadcastingOutput.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/multipleinput/output/BroadcastingOutput.java @@ -21,6 +21,7 @@ import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.OperatorChain; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; @@ -90,4 +91,11 @@ public void close() { output.close(); } } + + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) { + for (Output> output : outputs) { + output.emitRecordAttributes(recordAttributes); + } + } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/StreamingWithBacklogITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/StreamingWithBacklogITCase.java new file mode 100644 index 00000000000000..f2cd471b1858f7 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/StreamingWithBacklogITCase.java @@ -0,0 +1,265 @@ +/* + * 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; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.AggregateFunction; +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.hybrid.HybridSource; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.windowing.WindowFunction; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.CollectionUtil; + +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL_DURING_BACKLOG; +import static org.assertj.core.api.Assertions.assertThat; + +/** Integration test for streaming job with backlog. */ +public class StreamingWithBacklogITCase { + @Test + public void testKeyedAggregationWithBacklog() throws Exception { + final Configuration config = new Configuration(); + config.set(CHECKPOINTING_INTERVAL_DURING_BACKLOG, Duration.ZERO); + final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(config); + + List> expectedRes = new ArrayList<>(); + for (int i = 1; i <= 10000; ++i) { + expectedRes.add(Tuple2.of(0, (long) i)); + expectedRes.add(Tuple2.of(1, (long) i)); + } + + assertThat(runKeyedAggregation(env)).containsExactlyInAnyOrderElementsOf(expectedRes); + } + + @Test + public void testKeyedAggregationWithBacklogParallelismOne() throws Exception { + final Configuration config = new Configuration(); + config.set(CHECKPOINTING_INTERVAL_DURING_BACKLOG, Duration.ZERO); + final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(config); + env.setParallelism(1); + + List> backlogExpectedRes = new ArrayList<>(); + List> realTimeExpectedRes = new ArrayList<>(); + for (int i = 1; i <= 5000; ++i) { + backlogExpectedRes.add(Tuple2.of(0, (long) i)); + } + for (int i = 1; i <= 5000; ++i) { + backlogExpectedRes.add(Tuple2.of(1, (long) i)); + } + for (int i = 5001; i <= 10000; ++i) { + realTimeExpectedRes.add(Tuple2.of(0, (long) i)); + realTimeExpectedRes.add(Tuple2.of(1, (long) i)); + } + final List> result = runKeyedAggregation(env); + assertThat(getKeySwitchCnt(result.subList(0, 10000), x -> x.f0)).isEqualTo(1); + assertThat(result.subList(0, 10000)) + .containsExactlyInAnyOrderElementsOf(backlogExpectedRes); + assertThat(result.subList(10000, 20000)) + .containsExactlyInAnyOrderElementsOf(realTimeExpectedRes); + } + + @Test + public void testKeyedWindowedAggregationWithBacklog() throws Exception { + final Configuration config = new Configuration(); + config.set(CHECKPOINTING_INTERVAL_DURING_BACKLOG, Duration.ZERO); + final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(config); + + List> expectedRes = new ArrayList<>(); + for (int i = 1; i <= 5000; ++i) { + expectedRes.add(Tuple3.of(0, (long) i * 4 * 1000, 2L)); + expectedRes.add(Tuple3.of(1, (long) i * 4 * 1000, 2L)); + } + + assertThat(runKeyedWindowedAggregation(env)) + .containsExactlyInAnyOrderElementsOf(expectedRes); + } + + @Test + public void testKeyedWindowedAggregationWithBacklogParallelismOne() throws Exception { + final Configuration config = new Configuration(); + config.set(CHECKPOINTING_INTERVAL_DURING_BACKLOG, Duration.ZERO); + final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(config); + env.setParallelism(1); + + List> backlogExpectedRes = new ArrayList<>(); + List> realTimeExpectedRes = new ArrayList<>(); + + for (int i = 1; i <= 2499; ++i) { + backlogExpectedRes.add(Tuple3.of(0, (long) i * 4 * 1000, 2L)); + } + for (int i = 1; i <= 2499; ++i) { + backlogExpectedRes.add(Tuple3.of(1, (long) i * 4 * 1000, 2L)); + } + for (int i = 2500; i <= 5000; ++i) { + realTimeExpectedRes.add(Tuple3.of(0, (long) i * 4 * 1000, 2L)); + realTimeExpectedRes.add(Tuple3.of(1, (long) i * 4 * 1000, 2L)); + } + + final List> result = runKeyedWindowedAggregation(env); + assertThat(getKeySwitchCnt(result.subList(0, 4998), x -> x.f0)).isEqualTo(1); + assertThat(result.subList(0, 4998)).containsExactlyInAnyOrderElementsOf(backlogExpectedRes); + assertThat(result.subList(4998, 10000)) + .containsExactlyInAnyOrderElementsOf(realTimeExpectedRes); + } + + private List> runKeyedAggregation(StreamExecutionEnvironment env) + throws Exception { + final DataGeneratorSource> historicalData = + new DataGeneratorSource<>( + (GeneratorFunction>) + value -> new Tuple2<>(value.intValue() % 2, 1L), + 10000, + Types.TUPLE(Types.INT, Types.LONG)); + + final DataGeneratorSource> realTimeData = + new DataGeneratorSource<>( + (GeneratorFunction>) + value -> new Tuple2<>(value.intValue() % 2, 1L), + 10000, + Types.TUPLE(Types.INT, Types.LONG)); + + final HybridSource> source = + HybridSource.builder(historicalData).addSource(realTimeData).build(); + final SingleOutputStreamOperator> reduced = + env.fromSource(source, WatermarkStrategy.noWatermarks(), "source") + .returns(Types.TUPLE(Types.INT, Types.LONG)) + .setParallelism(1) + .keyBy(record -> record.f0) + .reduce( + (ReduceFunction>) + (value1, value2) -> + new Tuple2<>(value1.f0, value1.f1 + value2.f1)); + + try (final CloseableIterator> iter = reduced.executeAndCollect()) { + return CollectionUtil.iteratorToList(iter); + } + } + + private List> runKeyedWindowedAggregation( + StreamExecutionEnvironment env) throws Exception { + final int backlogCnt = 10000; + final DataGeneratorSource> historicalData = + new DataGeneratorSource<>( + (GeneratorFunction>) + value -> new Tuple3<>(value.intValue() % 2, value * 1000, 1L), + backlogCnt, + Types.TUPLE(Types.INT, Types.LONG)); + + final DataGeneratorSource> realTimeData = + new DataGeneratorSource<>( + (GeneratorFunction>) + value -> + new Tuple3<>( + value.intValue() % 2, + (value + backlogCnt) * 1000, + 1L), + 10000, + Types.TUPLE(Types.INT, Types.LONG)); + + final HybridSource> source = + HybridSource.builder(historicalData).addSource(realTimeData).build(); + final SingleOutputStreamOperator> output = + env.fromSource( + source, + WatermarkStrategy + .>forMonotonousTimestamps() + .withTimestampAssigner((event, timestamp) -> event.f1), + "source") + .setParallelism(1) + .returns(Types.TUPLE(Types.INT, Types.LONG, Types.LONG)) + .keyBy(record -> record.f0) + .window(TumblingEventTimeWindows.of(Time.seconds(4))) + .aggregate( + new AggregateFunction, Long, Long>() { + @Override + public Long createAccumulator() { + return 0L; + } + + @Override + public Long add( + Tuple3 value, Long accumulator) { + return accumulator + value.f2; + } + + @Override + public Long getResult(Long accumulator) { + return accumulator; + } + + @Override + public Long merge(Long a, Long b) { + return a + b; + } + }, + (WindowFunction< + Long, + Tuple3, + Integer, + TimeWindow>) + (key, window, input, out) -> { + for (Long i : input) { + out.collect(Tuple3.of(key, window.getEnd(), i)); + } + }) + .returns(Types.TUPLE(Types.INT, Types.LONG, Types.LONG)); + + List> expectedRes = new ArrayList<>(); + for (int i = 1; i <= 5000; ++i) { + expectedRes.add(Tuple3.of(0, (long) i * 4 * 1000, 2L)); + expectedRes.add(Tuple3.of(1, (long) i * 4 * 1000, 2L)); + } + try (final CloseableIterator> iter = + output.executeAndCollect()) { + return CollectionUtil.iteratorToList(iter); + } + } + + private int getKeySwitchCnt(List result, Function keySelector) { + final List keys = result.stream().map(keySelector).collect(Collectors.toList()); + int count = 0; + for (int i = 1; i < keys.size(); ++i) { + if (keys.get(i - 1) != keys.get(i)) { + count++; + } + } + return count; + } +}