public void setMaxParallelism(int vertexID, int maxParallelism) { if (getStreamNode(vertexID) != null) { getStreamNode(vertexID).setMaxParallelism(maxParallelism); } }
public void setResources(int vertexID, ResourceSpec minResources, ResourceSpec preferredResources) { if (getStreamNode(vertexID) != null) { getStreamNode(vertexID).setResources(minResources, preferredResources); } }
public void setBufferTimeout(Integer vertexID, long bufferTimeout) { if (getStreamNode(vertexID) != null) { getStreamNode(vertexID).setBufferTimeout(bufferTimeout); } }
public void setParallelism(Integer vertexID, int parallelism) { if (getStreamNode(vertexID) != null) { getStreamNode(vertexID).setParallelism(parallelism); } }
public <IN, OUT> void setOperator(Integer vertexID, StreamOperator<OUT> operatorObject) { getStreamNode(vertexID).setOperator(operatorObject); }
public void setInputFormat(Integer vertexID, InputFormat<?, ?> inputFormat) { getStreamNode(vertexID).setInputFormat(inputFormat); }
public List<StreamEdge> getStreamEdges(int sourceId, int targetId) { List<StreamEdge> result = new ArrayList<>(); for (StreamEdge edge : getStreamNode(sourceId).getOutEdges()) { if (edge.getTargetId() == targetId) { result.add(edge); } } if (result.isEmpty()) { throw new RuntimeException("No such edge in stream graph: " + sourceId + " -> " + targetId); } return result; }
public <T> void addOutputSelector(Integer vertexID, OutputSelector<T> outputSelector) { if (virtualPartitionNodes.containsKey(vertexID)) { addOutputSelector(virtualPartitionNodes.get(vertexID).f0, outputSelector); } else if (virtualSelectNodes.containsKey(vertexID)) { addOutputSelector(virtualSelectNodes.get(vertexID).f0, outputSelector); } else { getStreamNode(vertexID).addOutputSelector(outputSelector); if (LOG.isDebugEnabled()) { LOG.debug("Outputselector set for {}", vertexID); } } }
private String createChainedName(Integer vertexID, List<StreamEdge> chainedOutputs) { String operatorName = streamGraph.getStreamNode(vertexID).getOperatorName(); if (chainedOutputs.size() > 1) { List<String> outputChainedNames = new ArrayList<>(); for (StreamEdge chainable : chainedOutputs) { outputChainedNames.add(chainedNames.get(chainable.getTargetId())); } return operatorName + " -> (" + StringUtils.join(outputChainedNames, ", ") + ")"; } else if (chainedOutputs.size() == 1) { return operatorName + " -> " + chainedNames.get(chainedOutputs.get(0).getTargetId()); } else { return operatorName; } }
public void setOneInputStateKey(Integer vertexID, KeySelector<?, ?> keySelector, TypeSerializer<?> keySerializer) { StreamNode node = getStreamNode(vertexID); node.setStatePartitioner1(keySelector); node.setStateKeySerializer(keySerializer); }
public void setSerializersFrom(Integer from, Integer to) { StreamNode fromVertex = getStreamNode(from); StreamNode toVertex = getStreamNode(to); toVertex.setSerializerIn1(fromVertex.getTypeSerializerOut()); toVertex.setSerializerOut(fromVertex.getTypeSerializerIn1()); }
public <OUT> void setOutType(Integer vertexID, TypeInformation<OUT> outType) { getStreamNode(vertexID).setSerializerOut(outType.createSerializer(executionConfig)); }
private ResourceSpec createChainedPreferredResources(Integer vertexID, List<StreamEdge> chainedOutputs) { ResourceSpec preferredResources = streamGraph.getStreamNode(vertexID).getPreferredResources(); for (StreamEdge chainable : chainedOutputs) { preferredResources = preferredResources.merge(chainedPreferredResources.get(chainable.getTargetId())); } return preferredResources; }
private ResourceSpec createChainedMinResources(Integer vertexID, List<StreamEdge> chainedOutputs) { ResourceSpec minResources = streamGraph.getStreamNode(vertexID).getMinResources(); for (StreamEdge chainable : chainedOutputs) { minResources = minResources.merge(chainedMinResources.get(chainable.getTargetId())); } return minResources; }
public void setSerializers(Integer vertexID, TypeSerializer<?> in1, TypeSerializer<?> in2, TypeSerializer<?> out) { StreamNode vertex = getStreamNode(vertexID); vertex.setSerializerIn1(in1); vertex.setSerializerIn2(in2); vertex.setSerializerOut(out); }
public void setTwoInputStateKey(Integer vertexID, KeySelector<?, ?> keySelector1, KeySelector<?, ?> keySelector2, TypeSerializer<?> keySerializer) { StreamNode node = getStreamNode(vertexID); node.setStatePartitioner1(keySelector1); node.setStatePartitioner2(keySelector2); node.setStateKeySerializer(keySerializer); }
private void decorateNode(Integer vertexID, ObjectNode node) { StreamNode vertex = streamGraph.getStreamNode(vertexID); node.put(ID, vertexID); node.put(TYPE, vertex.getOperatorName()); if (streamGraph.getSourceIDs().contains(vertexID)) { node.put(PACT, "Data Source"); } else if (streamGraph.getSinkIDs().contains(vertexID)) { node.put(PACT, "Data Sink"); } else { node.put(PACT, "Operator"); } StreamOperator<?> operator = streamGraph.getStreamNode(vertexID).getOperator(); node.put(CONTENTS, vertex.getOperatorName()); node.put(PARALLELISM, streamGraph.getStreamNode(vertexID).getParallelism()); }
private static StreamOperator<?> getOperatorForDataStream(DataStream<?> dataStream) { StreamExecutionEnvironment env = dataStream.getExecutionEnvironment(); StreamGraph streamGraph = env.getStreamGraph(); return streamGraph.getStreamNode(dataStream.getId()).getOperator(); }
private static StreamOperator<?> getOperatorFromDataStream(DataStream<?> dataStream) { StreamExecutionEnvironment env = dataStream.getExecutionEnvironment(); StreamGraph streamGraph = env.getStreamGraph(); return streamGraph.getStreamNode(dataStream.getId()).getOperator(); }
/** * Tests that the KeyGroupStreamPartitioner are properly set up with the correct value of * maximum parallelism. */ @Test public void testSetupOfKeyGroupPartitioner() { int maxParallelism = 42; StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.getConfig().setMaxParallelism(maxParallelism); DataStream<Integer> source = env.fromElements(1, 2, 3); DataStream<Integer> keyedResult = source.keyBy(value -> value).map(new NoOpIntMap()); keyedResult.addSink(new DiscardingSink<>()); StreamGraph graph = env.getStreamGraph(); StreamNode keyedResultNode = graph.getStreamNode(keyedResult.getId()); StreamPartitioner<?> streamPartitioner = keyedResultNode.getInEdges().get(0).getPartitioner(); }