Skip to content

Commit

Permalink
[FLINK-18832][datastream] Add compatible check for blocking partition…
Browse files Browse the repository at this point in the history
… with buffer timeout

From the requirement it is no need to enable buffer timeout for batch jobs since the downstream can only consume data when the upstream finishes.
Furthermore the current implementation of BoundedBlockingSubpartition does not consider the concurrent issue from the flusher thread by enabling
buffer timeout. So it is nice to check this compatibility during job graph generation in advance and give a friendly message hint for users.

This closes apache#13209.
  • Loading branch information
zhijiangW committed Sep 7, 2020
1 parent 1f2d26f commit 13e0b35
Show file tree
Hide file tree
Showing 11 changed files with 88 additions and 61 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -83,6 +83,7 @@
import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit;
import org.apache.flink.streaming.api.graph.StreamGraph;
import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
import org.apache.flink.streaming.api.graph.StreamingJobGraphGenerator;
import org.apache.flink.streaming.api.operators.StreamSource;
import org.apache.flink.util.DynamicCodeLoadingException;
import org.apache.flink.util.ExceptionUtils;
Expand Down Expand Up @@ -128,9 +129,6 @@ public class StreamExecutionEnvironment {
/** The time characteristic that is used if none other is set. */
private static final TimeCharacteristic DEFAULT_TIME_CHARACTERISTIC = TimeCharacteristic.ProcessingTime;

/** The default buffer timeout (max delay of records in the network stack). */
private static final long DEFAULT_NETWORK_BUFFER_TIMEOUT = 100L;

/**
* The environment of the context (local by default, cluster if invoked through command line).
*/
Expand All @@ -152,7 +150,7 @@ public class StreamExecutionEnvironment {

protected final List<Transformation<?>> transformations = new ArrayList<>();

private long bufferTimeout = DEFAULT_NETWORK_BUFFER_TIMEOUT;
private long bufferTimeout = StreamingJobGraphGenerator.UNDEFINED_NETWORK_BUFFER_TIMEOUT;

protected boolean isChainingEnabled = true;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,14 +75,12 @@ public class StreamConfig implements Serializable {
private static final String VERTEX_NAME = "vertexID";
private static final String ITERATION_ID = "iterationId";
private static final String OUTPUT_SELECTOR_WRAPPER = "outputSelectorWrapper";
private static final String BUFFER_TIMEOUT = "bufferTimeout";
private static final String INPUTS = "inputs";
private static final String TYPE_SERIALIZER_OUT_1 = "typeSerializer_out";
private static final String TYPE_SERIALIZER_SIDEOUT_PREFIX = "typeSerializer_sideout_";
private static final String ITERATON_WAIT = "iterationWait";
private static final String NONCHAINED_OUTPUTS = "nonChainedOutputs";
private static final String EDGES_IN_ORDER = "edgesInOrder";
private static final String OUT_STREAM_EDGES = "outStreamEdges";
private static final String IN_STREAM_EDGES = "inStreamEdges";
private static final String OPERATOR_NAME = "operatorName";
private static final String OPERATOR_ID = "operatorID";
Expand All @@ -104,7 +102,6 @@ public class StreamConfig implements Serializable {
// Default Values
// ------------------------------------------------------------------------

private static final long DEFAULT_TIMEOUT = 100;
private static final CheckpointingMode DEFAULT_CHECKPOINTING_MODE = CheckpointingMode.EXACTLY_ONCE;

private static final double DEFAULT_MANAGED_MEMORY_FRACTION = 0.0;
Expand Down Expand Up @@ -238,14 +235,6 @@ public <T> TypeSerializer<T> getTypeSerializerIn(int index, ClassLoader cl) {
return (TypeSerializer<T>) ((NetworkInputConfig) inputs[index]).typeSerializer;
}

public void setBufferTimeout(long timeout) {
config.setLong(BUFFER_TIMEOUT, timeout);
}

public long getBufferTimeout() {
return config.getLong(BUFFER_TIMEOUT, DEFAULT_TIMEOUT);
}

@VisibleForTesting
public void setStreamOperator(StreamOperator<?> operator) {
setStreamOperatorFactory(SimpleOperatorFactory.of(operator));
Expand Down Expand Up @@ -374,23 +363,6 @@ public List<StreamEdge> getChainedOutputs(ClassLoader cl) {
}
}

public void setOutEdges(List<StreamEdge> outEdges) {
try {
InstantiationUtil.writeObjectToConfig(outEdges, this.config, OUT_STREAM_EDGES);
} catch (IOException e) {
throw new StreamTaskException("Cannot serialize outward edges.", e);
}
}

public List<StreamEdge> getOutEdges(ClassLoader cl) {
try {
List<StreamEdge> outEdges = InstantiationUtil.readObjectFromConfig(this.config, OUT_STREAM_EDGES, cl);
return outEdges == null ? new ArrayList<StreamEdge>() : outEdges;
} catch (Exception e) {
throw new StreamTaskException("Could not instantiate outputs.", e);
}
}

public void setInPhysicalEdges(List<StreamEdge> inEdges) {
try {
InstantiationUtil.writeObjectToConfig(inEdges, this.config, IN_STREAM_EDGES);
Expand Down Expand Up @@ -612,7 +584,6 @@ public String toString() {
catch (Exception e) {
builder.append("\nOperator: Missing");
}
builder.append("\nBuffer timeout: ").append(getBufferTimeout());
builder.append("\nState Monitoring: ").append(isCheckpointingEnabled());
if (isChainStart() && getChainedOutputs(cl).size() > 0) {
builder.append("\n\n\n---------------------\nChained task configs\n---------------------\n");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import java.util.List;
import java.util.Objects;

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

/**
Expand All @@ -38,6 +39,8 @@ public class StreamEdge implements Serializable {

private static final long serialVersionUID = 1L;

private static final long ALWAYS_FLUSH_BUFFER_TIMEOUT = 0L;

private final String edgeId;

private final int sourceId;
Expand Down Expand Up @@ -76,6 +79,8 @@ public class StreamEdge implements Serializable {

private final ShuffleMode shuffleMode;

private long bufferTimeout;

public StreamEdge(
StreamNode sourceVertex,
StreamNode targetVertex,
Expand All @@ -88,6 +93,7 @@ public StreamEdge(
sourceVertex,
targetVertex,
typeNumber,
ALWAYS_FLUSH_BUFFER_TIMEOUT,
selectedNames,
outputPartitioner,
outputTag,
Expand All @@ -103,9 +109,31 @@ public StreamEdge(
OutputTag outputTag,
ShuffleMode shuffleMode) {

this(
sourceVertex,
targetVertex,
typeNumber,
sourceVertex.getBufferTimeout(),
selectedNames,
outputPartitioner,
outputTag,
shuffleMode);
}

public StreamEdge(
StreamNode sourceVertex,
StreamNode targetVertex,
int typeNumber,
long bufferTimeout,
List<String> selectedNames,
StreamPartitioner<?> outputPartitioner,
OutputTag outputTag,
ShuffleMode shuffleMode) {

this.sourceId = sourceVertex.getId();
this.targetId = targetVertex.getId();
this.typeNumber = typeNumber;
this.bufferTimeout = bufferTimeout;
this.selectedNames = selectedNames;
this.outputPartitioner = outputPartitioner;
this.outputTag = outputTag;
Expand Down Expand Up @@ -147,6 +175,15 @@ public void setPartitioner(StreamPartitioner<?> partitioner) {
this.outputPartitioner = partitioner;
}

public void setBufferTimeout(long bufferTimeout) {
checkArgument(bufferTimeout >= -1);
this.bufferTimeout = bufferTimeout;
}

public long getBufferTimeout() {
return bufferTimeout;
}

@Override
public int hashCode() {
return Objects.hash(edgeId, outputTag);
Expand All @@ -170,6 +207,6 @@ public boolean equals(Object o) {
public String toString() {
return "(" + (sourceOperatorName + "-" + sourceId) + " -> " + (targetOperatorName + "-" + targetId)
+ ", typeNumber=" + typeNumber + ", selectedNames=" + selectedNames + ", outputPartitioner=" + outputPartitioner
+ ", outputTag=" + outputTag + ')';
+ ", bufferTimeout=" + bufferTimeout + ", outputTag=" + outputTag + ')';
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -104,9 +104,6 @@ public class StreamGraphGenerator {

public static final String DEFAULT_JOB_NAME = "Flink Streaming Job";

/** The default buffer timeout (max delay of records in the network stack). */
public static final long DEFAULT_NETWORK_BUFFER_TIMEOUT = 100L;

public static final String DEFAULT_SLOT_SHARING_GROUP = "default";

private final List<Transformation<?>> transformations;
Expand All @@ -127,7 +124,7 @@ public class StreamGraphGenerator {

private TimeCharacteristic timeCharacteristic = DEFAULT_TIME_CHARACTERISTIC;

private long defaultBufferTimeout = DEFAULT_NETWORK_BUFFER_TIMEOUT;
private long defaultBufferTimeout = StreamingJobGraphGenerator.UNDEFINED_NETWORK_BUFFER_TIMEOUT;

private String jobName = DEFAULT_JOB_NAME;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,6 +99,10 @@ public class StreamingJobGraphGenerator {

private static final int MANAGED_MEMORY_FRACTION_SCALE = 16;

private static final long DEFAULT_NETWORK_BUFFER_TIMEOUT = 100L;

public static final long UNDEFINED_NETWORK_BUFFER_TIMEOUT = -1L;

// ------------------------------------------------------------------------

public static JobGraph createJobGraph(StreamGraph streamGraph) {
Expand Down Expand Up @@ -318,13 +322,12 @@ private List<StreamEdge> createChain(Integer currentNodeId, int chainIndex, Oper
config.setChainStart();
config.setChainIndex(0);
config.setOperatorName(streamGraph.getStreamNode(currentNodeId).getOperatorName());
config.setOutEdgesInOrder(transitiveOutEdges);
config.setOutEdges(streamGraph.getStreamNode(currentNodeId).getOutEdges());

for (StreamEdge edge : transitiveOutEdges) {
connect(startNodeId, edge);
}

config.setOutEdgesInOrder(transitiveOutEdges);
config.setTransitiveChainedTaskConfigs(chainedConfigs.get(startNodeId));

} else {
Expand Down Expand Up @@ -469,7 +472,6 @@ private void setVertexConfig(Integer vertexID, StreamConfig config,
StreamNode vertex = streamGraph.getStreamNode(vertexID);

config.setVertexID(vertexID);
config.setBufferTimeout(vertex.getBufferTimeout());

config.setTypeSerializersIn(vertex.getTypeSerializersIn());
config.setTypeSerializerOut(vertex.getTypeSerializerOut());
Expand Down Expand Up @@ -570,6 +572,8 @@ private void connect(Integer headOfChain, StreamEdge edge) {
edge.getShuffleMode() + " is not supported yet.");
}

checkAndResetBufferTimeout(resultPartitionType, edge);

JobEdge jobEdge;
if (isPointwisePartitioner(partitioner)) {
jobEdge = downStreamVertex.connectNewDataSetAsInput(
Expand All @@ -591,6 +595,19 @@ private void connect(Integer headOfChain, StreamEdge edge) {
}
}

private void checkAndResetBufferTimeout(ResultPartitionType type, StreamEdge edge) {
long bufferTimeout = edge.getBufferTimeout();
if (type.isBlocking() && bufferTimeout != UNDEFINED_NETWORK_BUFFER_TIMEOUT) {
throw new UnsupportedOperationException(
"Blocking partition does not support buffer timeout " + bufferTimeout + " for src operator in edge "
+ edge.toString() + ". \nPlease either reset buffer timeout as -1 or use the non-blocking partition.");
}

if (type.isPipelined() && bufferTimeout == UNDEFINED_NETWORK_BUFFER_TIMEOUT) {
edge.setBufferTimeout(DEFAULT_NETWORK_BUFFER_TIMEOUT);
}
}

private static boolean isPointwisePartitioner(StreamPartitioner<?> partitioner) {
return partitioner instanceof ForwardPartitioner || partitioner instanceof RescalePartitioner;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -243,14 +243,4 @@ public boolean isCheckpointingEnabled() {
public CheckpointingMode getCheckpointMode() {
return streamConfig.getCheckpointMode();
}

/**
* Returns the buffer timeout of the job.
*
* @return buffer timeout (in milliseconds)
*/
public long getBufferTimeout() {
return streamConfig.getBufferTimeout();
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,6 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.OptionalLong;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
Expand Down Expand Up @@ -1145,7 +1144,6 @@ private static <OUT> List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>
Environment environment) {
List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> recordWriters = new ArrayList<>();
List<StreamEdge> outEdgesInOrder = configuration.getOutEdgesInOrder(environment.getUserClassLoader());
Map<Integer, StreamConfig> chainedConfigs = configuration.getTransitiveChainedTaskConfigsWithSelf(environment.getUserClassLoader());

for (int i = 0; i < outEdgesInOrder.size(); i++) {
StreamEdge edge = outEdgesInOrder.get(i);
Expand All @@ -1155,7 +1153,7 @@ private static <OUT> List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>
i,
environment,
environment.getTaskInfo().getTaskName(),
chainedConfigs.get(edge.getSourceId()).getBufferTimeout()));
edge.getBufferTimeout()));
}
return recordWriters;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -609,6 +609,32 @@ public void testShuffleModeUndefined() {
sourceAndMapVertex.getProducedDataSets().get(0).getResultType());
}

@Test(expected = UnsupportedOperationException.class)
public void testConflictShuffleModeWithBufferTimeout() {
testCompatibleShuffleModeWithBufferTimeout(ShuffleMode.BATCH);
}

@Test
public void testNormalShuffleModeWithBufferTimeout() {
testCompatibleShuffleModeWithBufferTimeout(ShuffleMode.PIPELINED);
}

private void testCompatibleShuffleModeWithBufferTimeout(ShuffleMode shuffleMode) {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setBufferTimeout(100);

DataStream<Integer> sourceDataStream = env.fromElements(1, 2, 3);
PartitionTransformation<Integer> transformation = new PartitionTransformation<>(
sourceDataStream.getTransformation(),
new RebalancePartitioner<>(),
shuffleMode);

DataStream<Integer> partitionStream = new DataStream<>(env, transformation);
partitionStream.map(value -> value).print();

StreamingJobGraphGenerator.createJobGraph(env.getStreamGraph());
}

/**
* Test iteration job, check slot sharing group and co-location group.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,6 @@ public class StreamConfigChainer<OWNER> {
private final OWNER owner;
private final StreamConfig headConfig;
private final Map<Integer, StreamConfig> chainedConfigs = new HashMap<>();
private final long bufferTimeout;

private StreamConfig tailConfig;
private int chainIndex = MAIN_NODE_ID;
Expand All @@ -58,7 +57,6 @@ public class StreamConfigChainer<OWNER> {
this.owner = checkNotNull(owner);
this.headConfig = checkNotNull(headConfig);
this.tailConfig = checkNotNull(headConfig);
this.bufferTimeout = headConfig.getBufferTimeout();

head(headOperatorID);
}
Expand All @@ -67,7 +65,6 @@ private void head(OperatorID headOperatorID) {
headConfig.setOperatorID(headOperatorID);
headConfig.setChainStart();
headConfig.setChainIndex(chainIndex);
headConfig.setBufferTimeout(bufferTimeout);
}

public <T> StreamConfigChainer<OWNER> chain(
Expand Down Expand Up @@ -145,7 +142,6 @@ public <IN, OUT> StreamConfigChainer<OWNER> chain(
tailConfig.setStateKeySerializer(inputSerializer);
}
tailConfig.setChainIndex(chainIndex);
tailConfig.setBufferTimeout(bufferTimeout);

chainedConfigs.put(chainIndex, tailConfig);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,6 @@ public class StreamTaskMailboxTestHarnessBuilder<OUT> {

protected long memorySize = 1024 * 1024;
protected int bufferSize = 1024;
protected long bufferTimeout = 0;
protected Configuration jobConfig = new Configuration();
protected Configuration taskConfig = new Configuration();
protected StreamConfig streamConfig = new StreamConfig(taskConfig);
Expand Down Expand Up @@ -125,7 +124,6 @@ public StreamTaskMailboxTestHarnessBuilder<OUT> addSourceInput(SourceOperatorFac
}

public StreamTaskMailboxTestHarness<OUT> build() throws Exception {
streamConfig.setBufferTimeout(bufferTimeout);

TestTaskStateManager taskStateManager = new TestTaskStateManager(localRecoveryConfig);
if (taskStateSnapshots != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -149,7 +149,6 @@ public StreamTaskTestHarness(
this.executionConfig = new ExecutionConfig();

streamConfig = new StreamConfig(taskConfig);
streamConfig.setBufferTimeout(0);

outputSerializer = outputType.createSerializer(executionConfig);
outputStreamRecordSerializer = new StreamElementSerializer<>(outputSerializer);
Expand Down

0 comments on commit 13e0b35

Please sign in to comment.