-
Notifications
You must be signed in to change notification settings - Fork 13.8k
[FLINK-23704][streaming] FLIP-27 sources are not generating LatencyMarkers #17105
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,10 +29,12 @@ | |
| import org.apache.flink.api.connector.source.SourceReaderContext; | ||
| import org.apache.flink.api.connector.source.SourceSplit; | ||
| import org.apache.flink.configuration.Configuration; | ||
| import org.apache.flink.configuration.MetricOptions; | ||
| import org.apache.flink.core.io.InputStatus; | ||
| import org.apache.flink.core.io.SimpleVersionedSerializer; | ||
| import org.apache.flink.metrics.groups.SourceReaderMetricGroup; | ||
| import org.apache.flink.runtime.io.AvailabilityProvider; | ||
| import org.apache.flink.runtime.jobgraph.OperatorID; | ||
| import org.apache.flink.runtime.metrics.groups.InternalSourceReaderMetricGroup; | ||
| import org.apache.flink.runtime.operators.coordination.OperatorEvent; | ||
| import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; | ||
|
|
@@ -49,6 +51,7 @@ | |
| import org.apache.flink.streaming.api.operators.util.SimpleVersionedListState; | ||
| import org.apache.flink.streaming.runtime.io.DataInputStatus; | ||
| import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput; | ||
| import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; | ||
| import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; | ||
| import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; | ||
| import org.apache.flink.streaming.runtime.tasks.StreamTask; | ||
|
|
@@ -57,11 +60,15 @@ | |
| import org.apache.flink.util.UserCodeClassLoader; | ||
| import org.apache.flink.util.function.FunctionWithException; | ||
|
|
||
| import javax.annotation.Nullable; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.List; | ||
| import java.util.concurrent.CompletableFuture; | ||
| import java.util.concurrent.ScheduledFuture; | ||
|
|
||
| import static org.apache.flink.util.Preconditions.checkNotNull; | ||
| import static org.apache.flink.util.Preconditions.checkState; | ||
|
|
||
| /** | ||
| * Base source operator only used for integrating the source reader which is proposed by FLIP-27. It | ||
|
|
@@ -149,6 +156,8 @@ private enum OperatingMode { | |
|
|
||
| private InternalSourceReaderMetricGroup sourceMetricGroup; | ||
|
|
||
| private LatencyMarkerEmitter<OUT> latencyMarerEmitter; | ||
|
|
||
| public SourceOperator( | ||
| FunctionWithException<SourceReaderContext, SourceReader<OUT, SplitT>, Exception> | ||
| readerFactory, | ||
|
|
@@ -282,6 +291,19 @@ public void open() throws Exception { | |
| watermarkStrategy, sourceMetricGroup); | ||
| } | ||
|
|
||
| latencyMarerEmitter = | ||
| new LatencyMarkerEmitter<>( | ||
| getProcessingTimeService(), | ||
| getExecutionConfig().isLatencyTrackingConfigured() | ||
| ? getExecutionConfig().getLatencyTrackingInterval() | ||
| : getContainingTask() | ||
| .getEnvironment() | ||
| .getTaskManagerInfo() | ||
| .getConfiguration() | ||
| .getLong(MetricOptions.LATENCY_INTERVAL), | ||
| getOperatorID(), | ||
| getRuntimeContext().getIndexOfThisSubtask()); | ||
|
|
||
| // restore the state if necessary. | ||
| final List<SplitT> splits = CollectionUtil.iterableToList(readerState.get()); | ||
| if (!splits.isEmpty()) { | ||
|
|
@@ -296,13 +318,17 @@ public void open() throws Exception { | |
| sourceReader.start(); | ||
|
|
||
| eventTimeLogic.startPeriodicWatermarkEmits(); | ||
| latencyMarerEmitter.startLatencyMarkerEmit(); | ||
| } | ||
|
|
||
| @Override | ||
| public void finish() throws Exception { | ||
| if (eventTimeLogic != null) { | ||
| eventTimeLogic.stopPeriodicWatermarkEmits(); | ||
| } | ||
| if (latencyMarerEmitter != null) { | ||
| latencyMarerEmitter.stopLatencyMarkerEmit(); | ||
| } | ||
| super.finish(); | ||
|
|
||
| finished.complete(null); | ||
|
|
@@ -348,6 +374,7 @@ private DataInputStatus emitNextNotReading(DataOutput<OUT> output) throws Except | |
| switch (operatingMode) { | ||
| case OUTPUT_NOT_INITIALIZED: | ||
| currentMainOutput = eventTimeLogic.createMainOutput(output); | ||
| latencyMarerEmitter.emitMainOutput(output); | ||
| lastInvokedOutput = output; | ||
| this.operatingMode = OperatingMode.READING; | ||
| return convertToInternalStatus(sourceReader.pollNext(currentMainOutput)); | ||
|
|
@@ -478,4 +505,79 @@ public void forceStop() { | |
| this.forcedStopFuture.complete(null); | ||
| } | ||
| } | ||
|
|
||
| private static class LatencyMarkerEmitter<OUT> { | ||
|
|
||
| private final ProcessingTimeService timeService; | ||
|
|
||
| private final long latencyTrackingInterval; | ||
|
|
||
| private final OperatorID operatorId; | ||
|
|
||
| private final int subtaskIndex; | ||
|
|
||
| @Nullable private DataOutput<OUT> currentMainOutput; | ||
|
|
||
| @Nullable private ScheduledFuture<?> latencyMarkerTimer; | ||
|
|
||
| public LatencyMarkerEmitter( | ||
| final ProcessingTimeService timeService, | ||
| long latencyTrackingInterval, | ||
| final OperatorID operatorId, | ||
| final int subtaskIndex) { | ||
| this.timeService = timeService; | ||
| this.latencyTrackingInterval = latencyTrackingInterval; | ||
| this.operatorId = operatorId; | ||
| this.subtaskIndex = subtaskIndex; | ||
| } | ||
|
|
||
| // ------------------------------------------------------------------------ | ||
|
|
||
| public void emitMainOutput(PushingAsyncDataInput.DataOutput<OUT> output) { | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. rename to |
||
| // At the moment, we assume only one output is ever created! | ||
| // This assumption is strict, currently, because many of the classes in this | ||
| // implementation | ||
| // do not support re-assigning the underlying output | ||
| checkState(currentMainOutput == null, "Main output has already been set."); | ||
| currentMainOutput = output; | ||
| } | ||
|
|
||
| public void startLatencyMarkerEmit() { | ||
| checkState( | ||
| latencyMarkerTimer == null, "Latency marker emitter has already been started"); | ||
| if (latencyTrackingInterval == 0) { | ||
| // a value of zero means not activated | ||
| return; | ||
| } | ||
| latencyMarkerTimer = | ||
| timeService.scheduleWithFixedDelay( | ||
| this::triggerLatencyMarkerEmit, 0L, latencyTrackingInterval); | ||
| } | ||
|
|
||
| public void stopLatencyMarkerEmit() { | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. rename to |
||
| if (latencyMarkerTimer != null) { | ||
| latencyMarkerTimer.cancel(false); | ||
| latencyMarkerTimer = null; | ||
| } | ||
| } | ||
|
|
||
| void triggerLatencyMarkerEmit(@SuppressWarnings("unused") long wallClockTimestamp) { | ||
| if (currentMainOutput != null) { | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. You can get rid of this if check (or replace is it with |
||
| try { | ||
| // ProcessingTimeService callbacks are executed under the | ||
| // checkpointing lock | ||
| currentMainOutput.emitLatencyMarker( | ||
| new LatencyMarker( | ||
| timeService.getCurrentProcessingTime(), | ||
| operatorId, | ||
| subtaskIndex)); | ||
| } catch (Throwable t) { | ||
| // we catch the Throwable here so that we don't trigger the | ||
| // processing | ||
| // timer services async exception handler | ||
| LOG.warn("Error while emitting latency marker.", t); | ||
| } | ||
| } | ||
| } | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we deduplicate this class with
LatencyMarksEmitter?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@pnowojski , this couldn't deduplicate with the
LatencyMarksEmitterinStreamSource, because this uses theDataOutputbutLatencyMarksEmitteruses theOutput.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
But why is that an issue? You could define a very simple helper interface inside the unified
LatencyMarksEmitter/LatencyMarkerEmitter:Use it in the unified class. And in
StreamSource/StreamOperatorinject it's implementation by just using a simple lambda functionoutput::emitLatencyMarkerto define it.