Skip to content

Commit

Permalink
[FLINK-18934][runtime] Idle stream does not advance watermark in conn…
Browse files Browse the repository at this point in the history
…ected stream

Watermark in the two and multi input operators is computed in operators. So far operators were unaware of the StreamStatus, therefore even if a whole input was IDLE it could still block increasing the Watermark.

This commit makes operators aware of the StreamStatus. The contract of the StreamStatus is that if a stream is IDLE it should not emit records nor watermarks.
  • Loading branch information
dawidwys committed May 25, 2021
1 parent 214ba68 commit 18a2a8a
Show file tree
Hide file tree
Showing 45 changed files with 801 additions and 83 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,87 @@
/*
* 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.api.common.eventtime;

import org.apache.flink.annotation.Internal;

import java.util.stream.IntStream;

import static org.apache.flink.util.Preconditions.checkArgument;

/**
* Represents combined value and status of a watermark for a set number of input partial watermarks.
*/
@Internal
public final class IndexedCombinedWatermarkStatus {
private final CombinedWatermarkStatus combinedWatermarkStatus;
private final CombinedWatermarkStatus.PartialWatermark[] partialWatermarks;

private IndexedCombinedWatermarkStatus(
CombinedWatermarkStatus combinedWatermarkStatus,
CombinedWatermarkStatus.PartialWatermark[] partialWatermarks) {
this.combinedWatermarkStatus = combinedWatermarkStatus;
this.partialWatermarks = partialWatermarks;
}

public static IndexedCombinedWatermarkStatus forInputsCount(int inputsCount) {
CombinedWatermarkStatus.PartialWatermark[] partialWatermarks =
IntStream.range(0, inputsCount)
.mapToObj(i -> new CombinedWatermarkStatus.PartialWatermark())
.toArray(CombinedWatermarkStatus.PartialWatermark[]::new);
CombinedWatermarkStatus combinedWatermarkStatus = new CombinedWatermarkStatus();
for (CombinedWatermarkStatus.PartialWatermark partialWatermark : partialWatermarks) {
combinedWatermarkStatus.add(partialWatermark);
}
return new IndexedCombinedWatermarkStatus(combinedWatermarkStatus, partialWatermarks);
}

/**
* Updates the value for the given partial watermark. Can update both the global idleness as
* well as the combined watermark value.
*
* @return true, if the combined watermark value changed. The global idleness needs to be
* checked separately via {@link #isIdle()}
*/
public boolean updateWatermark(int index, long timestamp) {
checkArgument(index < partialWatermarks.length);
partialWatermarks[index].setWatermark(timestamp);
return combinedWatermarkStatus.updateCombinedWatermark();
}

public long getCombinedWatermark() {
return combinedWatermarkStatus.getCombinedWatermark();
}

/**
* Updates the idleness for the given partial watermark. Can update both the global idleness as
* well as the combined watermark value.
*
* @return true, if the combined watermark value changed. The global idleness needs to be
* checked separately via {@link #isIdle()}
*/
public boolean updateStatus(int index, boolean idle) {
checkArgument(index < partialWatermarks.length);
partialWatermarks[index].setIdle(idle);
return combinedWatermarkStatus.updateCombinedWatermark();
}

public boolean isIdle() {
return combinedWatermarkStatus.isIdle();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
import org.apache.flink.streaming.runtime.tasks.StreamTask;
import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction;
Expand Down Expand Up @@ -129,6 +130,9 @@ private CollectorWrapper(Collector<OUT> inner) {
@Override
public void emitWatermark(Watermark mark) {}

@Override
public void emitStreamStatus(StreamStatus streamStatus) {}

@Override
public <X> void collect(OutputTag<X> outputTag, StreamRecord<X> record) {}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
import org.apache.flink.streaming.runtime.tasks.StreamTask;
import org.apache.flink.util.OutputTag;

Expand Down Expand Up @@ -93,6 +94,11 @@ public void processLatencyMarker(LatencyMarker latencyMarker) throws Exception {
operator.processLatencyMarker(latencyMarker);
}

@Override
public void emitStreamStatus(StreamStatus streamStatus) throws Exception {
operator.emitStreamStatus(streamStatus);
}

@Override
public void open() throws Exception {
operator.open();
Expand Down Expand Up @@ -201,6 +207,9 @@ private static class VoidOutput<T> implements Output<T> {
@Override
public void emitWatermark(Watermark mark) {}

@Override
public void emitStreamStatus(StreamStatus streamStatus) {}

@Override
public <X> void collect(OutputTag<X> outputTag, StreamRecord<X> record) {}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;

import javax.annotation.Nullable;

Expand Down Expand Up @@ -65,6 +66,11 @@ public void processLatencyMarker(LatencyMarker latencyMarker) throws Exception {
owner.reportOrForwardLatencyMarker(latencyMarker);
}

@Override
public void emitStreamStatus(StreamStatus streamStatus) throws Exception {
owner.emitStreamStatus(streamStatus, inputId);
}

@Override
public void setKeyContextElement(StreamRecord record) throws Exception {
owner.internalSetKeyContextElement(record, stateKeySelector);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.eventtime.IndexedCombinedWatermarkStatus;
import org.apache.flink.api.common.state.KeyedStateStore;
import org.apache.flink.api.common.state.State;
import org.apache.flink.api.common.state.StateDescriptor;
Expand Down Expand Up @@ -50,6 +51,7 @@
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
import org.apache.flink.streaming.runtime.tasks.StreamTask;
import org.apache.flink.streaming.util.LatencyStats;
Expand Down Expand Up @@ -88,7 +90,6 @@ public abstract class AbstractStreamOperator<OUT>
SetupableStreamOperator<OUT>,
CheckpointedStreamOperator,
Serializable {

private static final long serialVersionUID = 1L;

/** The logger used by the operator class and its subclasses. */
Expand All @@ -108,6 +109,8 @@ public abstract class AbstractStreamOperator<OUT>

protected transient Output<StreamRecord<OUT>> output;

private transient IndexedCombinedWatermarkStatus combinedWatermark;

/** The runtime context for UDFs. */
private transient StreamingRuntimeContext runtimeContext;

Expand Down Expand Up @@ -144,14 +147,6 @@ public abstract class AbstractStreamOperator<OUT>

protected transient ProcessingTimeService processingTimeService;

// ---------------- two-input operator watermarks ------------------

// We keep track of watermarks from both inputs, the combined input is the minimum
// Once the minimum advances we emit a new watermark for downstream operators
private long combinedWatermark = Long.MIN_VALUE;
private long input1Watermark = Long.MIN_VALUE;
private long input2Watermark = Long.MIN_VALUE;

// ------------------------------------------------------------------------
// Life Cycle
// ------------------------------------------------------------------------
Expand Down Expand Up @@ -183,6 +178,7 @@ public void setup(
this.output = output;
}

this.combinedWatermark = IndexedCombinedWatermarkStatus.forInputsCount(2);
try {
Configuration taskManagerConfig = environment.getTaskManagerInfo().getConfiguration();
int historySize = taskManagerConfig.getInteger(MetricOptions.LATENCY_HISTORY_SIZE);
Expand Down Expand Up @@ -628,24 +624,42 @@ public void processWatermark(Watermark mark) throws Exception {
output.emitWatermark(mark);
}

public void processWatermark1(Watermark mark) throws Exception {
input1Watermark = mark.getTimestamp();
long newMin = Math.min(input1Watermark, input2Watermark);
if (newMin > combinedWatermark) {
combinedWatermark = newMin;
processWatermark(new Watermark(combinedWatermark));
private void processWatermark(Watermark mark, int index) throws Exception {
if (combinedWatermark.updateWatermark(index, mark.getTimestamp())) {
processWatermark(new Watermark(combinedWatermark.getCombinedWatermark()));
}
}

public void processWatermark1(Watermark mark) throws Exception {
processWatermark(mark, 0);
}

public void processWatermark2(Watermark mark) throws Exception {
input2Watermark = mark.getTimestamp();
long newMin = Math.min(input1Watermark, input2Watermark);
if (newMin > combinedWatermark) {
combinedWatermark = newMin;
processWatermark(new Watermark(combinedWatermark));
processWatermark(mark, 1);
}

public final void emitStreamStatus(StreamStatus streamStatus) throws Exception {
output.emitStreamStatus(streamStatus);
}

private void emitStreamStatus(StreamStatus streamStatus, int index) throws Exception {
boolean wasIdle = combinedWatermark.isIdle();
if (combinedWatermark.updateStatus(index, streamStatus.isIdle())) {
processWatermark(new Watermark(combinedWatermark.getCombinedWatermark()));
}
if (wasIdle != combinedWatermark.isIdle()) {
output.emitStreamStatus(streamStatus);
}
}

public final void emitStreamStatus1(StreamStatus streamStatus) throws Exception {
emitStreamStatus(streamStatus, 0);
}

public final void emitStreamStatus2(StreamStatus streamStatus) throws Exception {
emitStreamStatus(streamStatus, 1);
}

@Override
public OperatorID getOperatorID() {
return config.getOperatorID();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.flink.annotation.Internal;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.eventtime.IndexedCombinedWatermarkStatus;
import org.apache.flink.api.common.state.KeyedStateStore;
import org.apache.flink.api.common.state.State;
import org.apache.flink.api.common.state.StateDescriptor;
Expand Down Expand Up @@ -50,6 +51,7 @@
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
import org.apache.flink.streaming.runtime.tasks.StreamTask;
import org.apache.flink.streaming.util.LatencyStats;
Expand All @@ -58,7 +60,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.Arrays;
import java.util.Locale;
import java.util.Optional;

Expand Down Expand Up @@ -89,7 +90,7 @@ public abstract class AbstractStreamOperatorV2<OUT>
private final ExecutionConfig executionConfig;
private final ClassLoader userCodeClassLoader;
private final CloseableRegistry cancelables;
private final long[] inputWatermarks;
private final IndexedCombinedWatermarkStatus combinedWatermark;

/** Metric group for the operator. */
protected final OperatorMetricGroup metrics;
Expand All @@ -100,13 +101,7 @@ public abstract class AbstractStreamOperatorV2<OUT>
private StreamOperatorStateHandler stateHandler;
private InternalTimeServiceManager<?> timeServiceManager;

// We keep track of watermarks from both inputs, the combined input is the minimum
// Once the minimum advances we emit a new watermark for downstream operators
private long combinedWatermark = Long.MIN_VALUE;

public AbstractStreamOperatorV2(StreamOperatorParameters<OUT> parameters, int numberOfInputs) {
inputWatermarks = new long[numberOfInputs];
Arrays.fill(inputWatermarks, Long.MIN_VALUE);
final Environment environment = parameters.getContainingTask().getEnvironment();
config = parameters.getStreamConfig();
CountingOutput<OUT> countingOutput;
Expand Down Expand Up @@ -146,6 +141,7 @@ public AbstractStreamOperatorV2(StreamOperatorParameters<OUT> parameters, int nu
executionConfig = parameters.getContainingTask().getExecutionConfig();
userCodeClassLoader = parameters.getContainingTask().getUserCodeClassLoader();
cancelables = parameters.getContainingTask().getCancelables();
this.combinedWatermark = IndexedCombinedWatermarkStatus.forInputsCount(numberOfInputs);

runtimeContext =
new StreamingRuntimeContext(
Expand Down Expand Up @@ -529,14 +525,18 @@ public void processWatermark(Watermark mark) throws Exception {
}

protected void reportWatermark(Watermark mark, int inputId) throws Exception {
inputWatermarks[inputId - 1] = mark.getTimestamp();
long newMin = mark.getTimestamp();
for (long inputWatermark : inputWatermarks) {
newMin = Math.min(inputWatermark, newMin);
if (combinedWatermark.updateWatermark(inputId - 1, mark.getTimestamp())) {
processWatermark(new Watermark(combinedWatermark.getCombinedWatermark()));
}
}

public final void emitStreamStatus(StreamStatus streamStatus, int inputId) throws Exception {
boolean wasIdle = combinedWatermark.isIdle();
if (combinedWatermark.updateStatus(inputId - 1, streamStatus.isIdle())) {
processWatermark(new Watermark(combinedWatermark.getCombinedWatermark()));
}
if (newMin > combinedWatermark) {
combinedWatermark = newMin;
processWatermark(new Watermark(combinedWatermark));
if (wasIdle != combinedWatermark.isIdle()) {
output.emitStreamStatus(streamStatus);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
import org.apache.flink.util.OutputTag;

/** Wrapping {@link Output} that updates metrics on the number of emitted elements. */
Expand All @@ -39,6 +40,11 @@ public void emitWatermark(Watermark mark) {
output.emitWatermark(mark);
}

@Override
public void emitStreamStatus(StreamStatus streamStatus) {
output.emitStreamStatus(streamStatus);
}

@Override
public void emitLatencyMarker(LatencyMarker latencyMarker) {
output.emitLatencyMarker(latencyMarker);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;

/** {@link Input} interface used in {@link MultipleInputStreamOperator}. */
@PublicEvolving
Expand Down Expand Up @@ -49,4 +50,6 @@ public interface Input<IN> {
void processLatencyMarker(LatencyMarker latencyMarker) throws Exception;

void setKeyContextElement(StreamRecord<IN> record) throws Exception;

void emitStreamStatus(StreamStatus streamStatus) throws Exception;
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
import org.apache.flink.util.Collector;
import org.apache.flink.util.OutputTag;

Expand All @@ -44,6 +45,8 @@ public interface Output<T> extends Collector<T> {
*/
void emitWatermark(Watermark mark);

void emitStreamStatus(StreamStatus streamStatus);

/**
* Emits a record to the side output identified by the given {@link OutputTag}.
*
Expand Down
Loading

0 comments on commit 18a2a8a

Please sign in to comment.