private boolean isChainable(StreamEdge edge, boolean isChainingEnabled, StreamGraph streamGraph) { StreamNode upStreamVertex = streamGraph.getSourceVertex(edge); StreamNode downStreamVertex = streamGraph.getTargetVertex(edge); StreamOperator<?> headOperator = upStreamVertex.getOperator(); StreamOperator<?> outOperator = downStreamVertex.getOperator(); return downStreamVertex.getInEdges().size() == 1 && outOperator != null && headOperator != null && upStreamVertex.isSameSlotSharingGroup(downStreamVertex) && outOperator.getChainingStrategy() == ChainingStrategy.ALWAYS && (headOperator.getChainingStrategy() == ChainingStrategy.HEAD || headOperator.getChainingStrategy() == ChainingStrategy.ALWAYS) && (edge.getPartitioner() instanceof ForwardPartitioner) && upStreamVertex.getParallelism() == downStreamVertex.getParallelism() && isChainingEnabled; } }
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()); }
for (StreamEdge outEdge : node.getOutEdges()) { if (isChainable(outEdge, isChainingEnabled, streamGraph)) { for (StreamEdge inEdge : node.getInEdges()) { byte[] otherHash = hashes.get(inEdge.getSourceId()); if (node.getOperator() instanceof AbstractUdfStreamOperator) { udfClassName = ((AbstractUdfStreamOperator<?, ?>) node.getOperator()) .getUserFunction().getClass().getName(); "'" + node.toString() + "' {id: " + node.getId() + ", " + "parallelism: " + node.getParallelism() + ", " + "user function: " + udfClassName + "}");
public StreamEdge(StreamNode sourceVertex, StreamNode targetVertex, int typeNumber, List<String> selectedNames, StreamPartitioner<?> outputPartitioner, OutputTag outputTag) { this.sourceId = sourceVertex.getId(); this.targetId = targetVertex.getId(); this.typeNumber = typeNumber; this.selectedNames = selectedNames; this.outputPartitioner = outputPartitioner; this.outputTag = outputTag; this.sourceOperatorName = sourceVertex.getOperatorName(); this.targetOperatorName = targetVertex.getOperatorName(); this.edgeId = sourceVertex + "_" + targetVertex + "_" + typeNumber + "_" + selectedNames + "_" + outputPartitioner; }
private void removeVertex(StreamNode toRemove) { Set<StreamEdge> edgesToRemove = new HashSet<>(); edgesToRemove.addAll(toRemove.getInEdges()); edgesToRemove.addAll(toRemove.getOutEdges()); for (StreamEdge edge : edgesToRemove) { removeEdge(edge); } streamNodes.remove(toRemove.getId()); }
public Set<Tuple2<Integer, StreamOperator<?>>> getOperators() { Set<Tuple2<Integer, StreamOperator<?>>> operatorSet = new HashSet<>(); for (StreamNode vertex : streamNodes.values()) { operatorSet.add(new Tuple2<Integer, StreamOperator<?>>(vertex.getId(), vertex .getOperator())); } return operatorSet; }
config.setBufferTimeout(vertex.getBufferTimeout()); config.setTypeSerializerIn1(vertex.getTypeSerializerIn1()); config.setTypeSerializerIn2(vertex.getTypeSerializerIn2()); config.setTypeSerializerOut(vertex.getTypeSerializerOut()); config.setStreamOperator(vertex.getOperator()); config.setOutputSelectors(vertex.getOutputSelectors()); config.setStatePartitioner(0, vertex.getStatePartitioner1()); config.setStatePartitioner(1, vertex.getStatePartitioner2()); config.setStateKeySerializer(vertex.getStateKeySerializer()); Class<? extends AbstractInvokable> vertexClass = vertex.getJobVertexClass();
String uid = streamNode.getTransformationUID(); nodeIdMap.put(streamNode.getId(), uid); StreamNodeProperty streamNodeProperty = new StreamNodeProperty(uid); uidToNodeProperties.put(uid, streamNodeProperty); streamNodeProperty.setName(getNodeName(streamNode.getOperatorName())); if (streamNode.getInEdges().isEmpty()) { streamNodeProperty.setPact("Source"); } else if (streamNode.getOutEdges().isEmpty()) { streamNodeProperty.setPact("Sink"); } else { streamNodeProperty.setPact("Operator"); streamNodeProperty.setName(getNodeName(streamNode.getOperatorName())); streamNodeProperty.setMaxParallelism(maxParallelism); if (streamNode.getParallelism() > 0) { streamNodeProperty.setParallelism(streamNode.getParallelism()); ResourceSpec minResourceSpec = streamNode.getMinResources(); Preconditions.checkArgument(minResourceSpec != null, "resource can not be null."); streamNodeProperty.setCpuCores(minResourceSpec.getCpuCores()); for (StreamEdge edge : source.getOutEdges()) { edgeCounts.putIfAbsent(edge.getTargetId(), 0); int index = edgeCounts.put(edge.getTargetId(), 1 + edgeCounts.get(edge.getTargetId()));
StreamNode itSink = sourceSinkPair.f1; assertEquals(4, itSource.getOutEdges().size()); assertEquals(3, itSink.getInEdges().size()); assertEquals(itSource.getParallelism(), itSink.getParallelism()); for (StreamEdge edge : itSource.getOutEdges()) { if (graph.getTargetVertex(edge).getOperatorName().equals("IterRebalanceMap")) { assertTrue(edge.getPartitioner() instanceof RebalancePartitioner); } else if (graph.getTargetVertex(edge).getOperatorName().equals("IterForwardMap")) { assertTrue(edge.getPartitioner() instanceof ForwardPartitioner); for (StreamEdge edge : itSink.getInEdges()) { if (graph.getStreamNode(edge.getSourceId()).getOperatorName().equals("ParallelizeMapShuffle")) { assertTrue(edge.getPartitioner() instanceof ShufflePartitioner); if (graph.getStreamNode(edge.getSourceId()).getOperatorName().equals("ParallelizeMapForward")) { assertTrue(edge.getPartitioner() instanceof ForwardPartitioner); if (graph.getStreamNode(edge.getSourceId()).getOperatorName().equals("EvenOddSourceMap")) { assertTrue(edge.getPartitioner() instanceof ForwardPartitioner); assertTrue(edge.getSelectedNames().contains("even"));
if (streamNode.getInputFormat() != null) { jobVertex = new InputFormatVertex( chainedNames.get(streamNodeId), userDefinedChainedOperatorVertexIds); TaskConfig taskConfig = new TaskConfig(jobVertex.getConfiguration()); taskConfig.setStubWrapper(new UserCodeObjectWrapper<Object>(streamNode.getInputFormat())); } else { jobVertex = new JobVertex( jobVertex.setInvokableClass(streamNode.getJobVertexClass()); int parallelism = streamNode.getParallelism(); jobVertex.setMaxParallelism(streamNode.getMaxParallelism());
private void removeEdge(StreamEdge edge) { getSourceVertex(edge).getOutEdges().remove(edge); getTargetVertex(edge).getInEdges().remove(edge); }
List<StreamEdge> nonChainableOutputs = new ArrayList<StreamEdge>(); for (StreamEdge outEdge : streamGraph.getStreamNode(currentNodeId).getOutEdges()) { if (isChainable(outEdge, streamGraph)) { chainableOutputs.add(outEdge); config.setOperatorName(streamGraph.getStreamNode(currentNodeId).getOperatorName()); config.setOutEdgesInOrder(transitiveOutEdges); config.setOutEdges(streamGraph.getStreamNode(currentNodeId).getOutEdges()); config.setOperatorName(node.getOperatorName()); chainedConfigs.get(startNodeId).put(currentNodeId, config);
assertTrue(graph.getStreamNode(rebalanceMap.getId()).getInEdges().get(0).getPartitioner() instanceof RebalancePartitioner); assertTrue(graph.getStreamNode(broadcastMap.getId()).getInEdges().get(0).getPartitioner() instanceof BroadcastPartitioner); assertEquals(rebalanceMap.getId(), graph.getSourceVertex(graph.getStreamNode(broadcastMap.getId()).getInEdges().get(0)).getId()); assertTrue(graph.getStreamNode(map1Operator.getId()).getOutEdges().get(0).getPartitioner() instanceof BroadcastPartitioner); assertTrue(graph.getStreamNode(map1Operator.getId()).getOutEdges().get(0).getSelectedNames().get(0).equals("even")); assertTrue(graph.getStreamNode(map1Operator.getId()).getOutputSelectors().contains(selector1)); assertTrue(graph.getStreamNode(map2Operator.getId()).getOutEdges().get(0).getPartitioner() instanceof GlobalPartitioner); assertTrue(graph.getStreamNode(map2Operator.getId()).getOutEdges().get(0).getSelectedNames().get(0).equals("odd")); assertTrue(graph.getStreamNode(map2Operator.getId()).getOutputSelectors().contains(selector2)); assertTrue(graph.getStreamNode(map3Operator.getId()).getOutEdges().get(0).getPartitioner() instanceof ShufflePartitioner); assertTrue(graph.getStreamNode(map3Operator.getId()).getOutEdges().get(0).getSelectedNames().get(0).equals("even")); assertTrue(graph.getStreamNode(map3Operator.getId()).getOutputSelectors().contains(selector3));
String userSpecifiedHash = node.getTransformationUID(); for (StreamEdge inEdge : node.getInEdges()) { byte[] hash = generateDeterministicHash(node, hasher, hashes, isChainingEnabled, streamGraph); if (hashes.put(node.getId(), hash) != null) { if (hashes.put(node.getId(), hash) != null) {
switch (node.getOperatorName()) { assertEquals(77L, node.getBufferTimeout().longValue()); break; case "B": assertEquals(0L, node.getBufferTimeout().longValue()); break; case "C": assertEquals(12L, node.getBufferTimeout().longValue()); break; case "D": assertEquals(77L, node.getBufferTimeout().longValue()); break; default: assertTrue(node.getOperator() instanceof StreamSource);
assertTrue(env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getStatePartitioner1() == null); assertTrue(env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getInEdges().get(0).getPartitioner() instanceof ForwardPartitioner); assertNotNull(env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getStatePartitioner1()); assertNotNull(env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getStateKeySerializer()); assertNotNull(env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getStateKeySerializer()); assertEquals(key1, env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getStatePartitioner1()); assertTrue(env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getInEdges().get(0).getPartitioner() instanceof KeyGroupStreamPartitioner); assertTrue(env.getStreamGraph().getStreamNode(sink3.getTransformation().getId()).getStatePartitioner1() != null); assertEquals(key2, env.getStreamGraph().getStreamNode(sink3.getTransformation().getId()).getStatePartitioner1()); assertTrue(env.getStreamGraph().getStreamNode(sink3.getTransformation().getId()).getInEdges().get(0).getPartitioner() instanceof KeyGroupStreamPartitioner);
@Override public Map<Integer, byte[]> traverseStreamGraphAndGenerateHashes(StreamGraph streamGraph) { HashMap<Integer, byte[]> hashResult = new HashMap<>(); for (StreamNode streamNode : streamGraph.getStreamNodes()) { String userHash = streamNode.getUserHash(); if (null != userHash) { hashResult.put(streamNode.getId(), StringUtils.hexStringToByte(userHash)); } } return hashResult; } }
if (partitioner == null && upstreamNode.getParallelism() == downstreamNode.getParallelism()) { partitioner = new ForwardPartitioner<Object>(); } else if (partitioner == null) { 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."); getStreamNode(edge.getSourceId()).addOutEdge(edge); getStreamNode(edge.getTargetId()).addInEdge(edge);
public void apple(StreamNode node) { // CONSIDER: find a better way to identify transformation with StreamNode, so that we can better // detect mismatch between JSON and stream graph. if (node != null) { node.setParallelism(parallelism); StreamNodeUtil.setMaxParallelism(node, maxParallelism); ResourceSpec.Builder builder = ResourceSpec.newBuilder() .setCpuCores(cpuCores) .setHeapMemoryInMB(heapMemoryInMB) .setDirectMemoryInMB(directMemoryInMB) .setNativeMemoryInMB(nativeMemoryInMB); if (gpuLoad > 0) { builder.setGPUResource(gpuLoad); } if (otherResources != null) { for (Map.Entry<String, Double> entry : otherResources.entrySet()) { builder.addExtendedResource(new CommonExtendedResource(entry.getKey(), entry.getValue())); } } if (floatingManagedMemoryInMB > 0) { builder.addExtendedResource(new CommonExtendedResource(ResourceSpec.FLOATING_MANAGED_MEMORY_NAME, floatingManagedMemoryInMB)); } if (node.getMinResources().getExtendedResources().containsKey(ResourceSpec.MANAGED_MEMORY_NAME)) { builder.addExtendedResource(new CommonExtendedResource(ResourceSpec.MANAGED_MEMORY_NAME, node.getMinResources().getExtendedResources().get(ResourceSpec.MANAGED_MEMORY_NAME).getValue())); } ResourceSpec resourceSpec = builder.build(); node.setResources(resourceSpec, resourceSpec); } }
|| Collections.disjoint(vertex.getInEdges(), toVisit)) { node.put(PREDECESSORS, inputs); for (StreamEdge inEdge : vertex.getInEdges()) { int inputID = inEdge.getSourceId(); } else { Integer iterationHead = -1; for (StreamEdge inEdge : vertex.getInEdges()) { int operator = inEdge.getSourceId(); obj.put(ID, iterationHead); obj.put(PACT, "IterativeDataStream"); obj.put(PARALLELISM, streamGraph.getStreamNode(iterationHead).getParallelism()); obj.put(CONTENTS, "Stream Iteration"); ArrayNode iterationInputs = mapper.createArrayNode();