Skip to content

Commit

Permalink
[FLINK-25199][network] Make sure StreamEdges are unique
Browse files Browse the repository at this point in the history
Previously, if there was a node that was self-unioned with itself,
it was creating a situation with two identical StreamEdges. Both
with the same partitioning, from the same source node to the same
target node.

This was causing issues when constructing output collectors and
picking the correct RecordWriters, as StreamTask was not able to
uniquely identify given StreamEdge and was assigning the same
RecordWriter to both of the edges. As a result all stream elements
were sent twice through the same RecordWriter. It was actually pretty
harmless apart of calculating the combined watermark downstream,
since all watermarks were always comming just from one single
edge/inputgate, and the unused edges were always stuck with
min watermark.

As a solution we are making sure that StreamEdges are unique
by introducing a uniqueId field, incremented for every pair
of StreamEdges connecting the same node.
  • Loading branch information
pnowojski committed Jan 21, 2022
1 parent 8c8e658 commit 5192fd7
Show file tree
Hide file tree
Showing 4 changed files with 123 additions and 44 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,15 @@ public class StreamEdge implements Serializable {
private final int sourceId;
private final int targetId;

/**
* Note that this field doesn't have to be unique among all {@link StreamEdge}s. It's enough if
* this field ensures that all logical instances of {@link StreamEdge} are unique, and {@link
* #hashCode()} are different and {@link #equals(Object)} returns false, for every possible pair
* of {@link StreamEdge}. Especially among two different {@link StreamEdge}s that are connecting
* the same pair of nodes.
*/
private final int uniqueId;

/** The type number of the input for co-tasks. */
private final int typeNumber;
/** The side-output tag (if any) of this {@link StreamEdge}. */
Expand Down Expand Up @@ -78,7 +87,8 @@ public StreamEdge(
ALWAYS_FLUSH_BUFFER_TIMEOUT,
outputPartitioner,
outputTag,
StreamExchangeMode.UNDEFINED);
StreamExchangeMode.UNDEFINED,
0);
}

public StreamEdge(
Expand All @@ -87,7 +97,8 @@ public StreamEdge(
int typeNumber,
StreamPartitioner<?> outputPartitioner,
OutputTag outputTag,
StreamExchangeMode exchangeMode) {
StreamExchangeMode exchangeMode,
int uniqueId) {

this(
sourceVertex,
Expand All @@ -96,7 +107,8 @@ public StreamEdge(
sourceVertex.getBufferTimeout(),
outputPartitioner,
outputTag,
exchangeMode);
exchangeMode,
uniqueId);
}

public StreamEdge(
Expand All @@ -106,10 +118,12 @@ public StreamEdge(
long bufferTimeout,
StreamPartitioner<?> outputPartitioner,
OutputTag outputTag,
StreamExchangeMode exchangeMode) {
StreamExchangeMode exchangeMode,
int uniqueId) {

this.sourceId = sourceVertex.getId();
this.targetId = targetVertex.getId();
this.uniqueId = uniqueId;
this.typeNumber = typeNumber;
this.bufferTimeout = bufferTimeout;
this.outputPartitioner = outputPartitioner;
Expand All @@ -118,7 +132,15 @@ public StreamEdge(
this.targetOperatorName = targetVertex.getOperatorName();
this.exchangeMode = checkNotNull(exchangeMode);
this.edgeId =
sourceVertex + "_" + targetVertex + "_" + typeNumber + "_" + outputPartitioner;
sourceVertex
+ "_"
+ targetVertex
+ "_"
+ typeNumber
+ "_"
+ outputPartitioner
+ "_"
+ uniqueId;
}

public int getSourceId() {
Expand Down Expand Up @@ -200,6 +222,8 @@ public String toString() {
+ bufferTimeout
+ ", outputTag="
+ outputTag
+ ", uniqueId="
+ uniqueId
+ ')';
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -643,50 +643,75 @@ private void addEdgeInternal(
outputTag,
exchangeMode);
} else {
StreamNode upstreamNode = getStreamNode(upStreamVertexID);
StreamNode downstreamNode = getStreamNode(downStreamVertexID);

// If no partitioner was specified and the parallelism of upstream and downstream
// operator matches use forward partitioning, use rebalance otherwise.
if (partitioner == null
&& upstreamNode.getParallelism() == downstreamNode.getParallelism()) {
partitioner = new ForwardPartitioner<Object>();
} else if (partitioner == null) {
partitioner = new RebalancePartitioner<Object>();
}
createActualEdge(
upStreamVertexID,
downStreamVertexID,
typeNumber,
partitioner,
outputTag,
exchangeMode);
}
}

if (partitioner instanceof ForwardPartitioner) {
if (upstreamNode.getParallelism() != downstreamNode.getParallelism()) {
throw new UnsupportedOperationException(
"Forward partitioning does not allow "
+ "change of parallelism. Upstream operation: "
+ upstreamNode
+ " parallelism: "
+ upstreamNode.getParallelism()
+ ", downstream operation: "
+ downstreamNode
+ " parallelism: "
+ downstreamNode.getParallelism()
+ " You must use another partitioning strategy, such as broadcast, rebalance, shuffle or global.");
}
}
private void createActualEdge(
Integer upStreamVertexID,
Integer downStreamVertexID,
int typeNumber,
StreamPartitioner<?> partitioner,
OutputTag outputTag,
StreamExchangeMode exchangeMode) {
StreamNode upstreamNode = getStreamNode(upStreamVertexID);
StreamNode downstreamNode = getStreamNode(downStreamVertexID);

// If no partitioner was specified and the parallelism of upstream and downstream
// operator matches use forward partitioning, use rebalance otherwise.
if (partitioner == null
&& upstreamNode.getParallelism() == downstreamNode.getParallelism()) {
partitioner = new ForwardPartitioner<Object>();
} else if (partitioner == null) {
partitioner = new RebalancePartitioner<Object>();
}

if (exchangeMode == null) {
exchangeMode = StreamExchangeMode.UNDEFINED;
if (partitioner instanceof ForwardPartitioner) {
if (upstreamNode.getParallelism() != downstreamNode.getParallelism()) {
throw new UnsupportedOperationException(
"Forward partitioning does not allow "
+ "change of parallelism. Upstream operation: "
+ upstreamNode
+ " parallelism: "
+ upstreamNode.getParallelism()
+ ", downstream operation: "
+ downstreamNode
+ " parallelism: "
+ downstreamNode.getParallelism()
+ " You must use another partitioning strategy, such as broadcast, rebalance, shuffle or global.");
}
}

StreamEdge edge =
new StreamEdge(
upstreamNode,
downstreamNode,
typeNumber,
partitioner,
outputTag,
exchangeMode);

getStreamNode(edge.getSourceId()).addOutEdge(edge);
getStreamNode(edge.getTargetId()).addInEdge(edge);
if (exchangeMode == null) {
exchangeMode = StreamExchangeMode.UNDEFINED;
}

/**
* Just make sure that {@link StreamEdge} connecting same nodes (for example as a result of
* self unioning a {@link DataStream}) are distinct and unique. Otherwise it would be
* difficult on the {@link StreamTask} to assign {@link RecordWriter}s to correct {@link
* StreamEdge}.
*/
int uniqueId = getStreamEdges(upstreamNode.getId(), downstreamNode.getId()).size();

StreamEdge edge =
new StreamEdge(
upstreamNode,
downstreamNode,
typeNumber,
partitioner,
outputTag,
exchangeMode,
uniqueId);

getStreamNode(edge.getSourceId()).addOutEdge(edge);
getStreamNode(edge.getTargetId()).addInEdge(edge);
}

public void setParallelism(Integer vertexID, int parallelism) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@
import java.util.Set;

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

/** Class representing the operators in the streaming programs, with all their properties. */
@Internal
Expand Down Expand Up @@ -122,6 +123,11 @@ public StreamNode(
}

public void addInEdge(StreamEdge inEdge) {
checkState(
outEdges.stream().noneMatch(inEdge::equals),
"Adding not unique edge = %s to existing outEdges = %s",
inEdge,
inEdges);
if (inEdge.getTargetId() != getId()) {
throw new IllegalArgumentException("Destination id doesn't match the StreamNode id");
} else {
Expand All @@ -130,6 +136,11 @@ public void addInEdge(StreamEdge inEdge) {
}

public void addOutEdge(StreamEdge outEdge) {
checkState(
outEdges.stream().noneMatch(outEdge::equals),
"Adding not unique edge = %s to existing outEdges = %s",
outEdge,
outEdges);
if (outEdge.getSourceId() != getId()) {
throw new IllegalArgumentException("Source id doesn't match the StreamNode id");
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -159,6 +159,25 @@ public void testWatermarkPropagation() throws Exception {
}
}

@Test
public void testSelfUnionWatermarkPropagation() throws Exception {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setParallelism(1);
DataStream<Integer> dataStream1 = env.fromElements(1, 2, 3);

dataStream1
.union(dataStream1)
.transform(
"Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new CustomOperator(false))
.addSink(new DiscardingSink<>());
env.execute();

assertEquals(
Watermark.MAX_WATERMARK,
CustomOperator.finalWatermarks[0].get(
CustomOperator.finalWatermarks[0].size() - 1));
}

@Test
public void testWatermarkPropagationNoFinalWatermarkOnStop() throws Exception {

Expand Down

0 comments on commit 5192fd7

Please sign in to comment.