private Set<ProcessorNode> initialProcessors(List<ProcessorNode> group) { Set<ProcessorNode> nodes = new HashSet<>(); Set<ProcessorNode> curSet = new HashSet<>(group); for (ProcessorNode node : group) { for (Node parent : parentNodes(node)) { if (!(parent instanceof ProcessorNode) || !curSet.contains(parent)) { nodes.add(node); } } } return nodes; } }
Node parentNode(Node curNode) { Set<Node> parentNode = parentNodes(curNode); if (parentNode.size() > 1) { throw new IllegalArgumentException("Node " + curNode + " has more than one parent node."); } if (parentNode.isEmpty()) { throw new IllegalArgumentException("Node " + curNode + " has no parent."); } return parentNode.iterator().next(); }
private Set<Window<?, ?>> getWindowParams(Set<ProcessorNode> initialProcessors) { Set<WindowNode> windowNodes = new HashSet<>(); Set<Node> parents; for (ProcessorNode processorNode : initialProcessors) { parents = parentNodes(processorNode); for (Node node : parents) { if (windowInfo.containsKey(node)) { windowNodes.add(windowInfo.get(node)); } } } return windowNodes.stream().map(WindowNode::getWindowParams).collect(Collectors.toSet()); }
private Set<Node> parentNodes(Node curNode) { Set<Node> nodes = new HashSet<>(); for (Node parent : StreamUtil.<Node>getParents(graph, curNode)) { if (parent instanceof ProcessorNode || parent instanceof SpoutNode) { nodes.add(parent); } else { nodes.addAll(parentNodes(parent)); } } return nodes; }
private Set<String> getWindowedParentStreams(ProcessorNode processorNode) { Set<String> res = new HashSet<>(); for (Node parent : parentNodes(processorNode)) { if (parent instanceof ProcessorNode && parent.isWindowed()) { res.addAll(parent.getOutputStreams()); } } return res; }
private void updateWindowInfo(WindowNode windowNode) { for (Node parent : parentNodes(windowNode)) { windowInfo.put(parent, windowNode); } String tsField = windowNode.getWindowParams().getTimestampField(); if (tsField != null) { if (timestampFieldName != null && !tsField.equals(timestampFieldName)) { throw new IllegalArgumentException("Cannot set different timestamp field names"); } timestampFieldName = tsField; } }
private void updateNodeGroupingInfo(PartitionNode partitionNode) { if (partitionNode.getGroupingInfo() != null) { for (Node parent : parentNodes(partitionNode)) { for (String parentStream : partitionNode.getParentStreams(parent)) { nodeGroupingInfo.put(parent, parentStream, partitionNode.getGroupingInfo()); } } } }
private Multimap<String, ProcessorNode> wireBolt(List<ProcessorNode> group, BoltDeclarer boltDeclarer, Set<ProcessorNode> initialProcessors) { LOG.debug("Wiring bolt with boltDeclarer {}, group {}, initialProcessors {}, nodeGroupingInfo {}", boltDeclarer, group, initialProcessors, nodeGroupingInfo); Multimap<String, ProcessorNode> streamToInitialProcessor = ArrayListMultimap.create(); Set<ProcessorNode> curSet = new HashSet<>(group); for (ProcessorNode curNode : initialProcessors) { for (Node parent : parentNodes(curNode)) { if (curSet.contains(parent)) { LOG.debug("Parent {} of curNode {} is in group {}", parent, curNode, group); } else { for (String stream : curNode.getParentStreams(parent)) { declareGrouping(boltDeclarer, parent, stream, nodeGroupingInfo.get(parent, stream)); // put global stream id for spouts if (parent.getComponentId().startsWith("spout")) { stream = parent.getComponentId() + stream; } else { // subscribe to parent's punctuation stream String punctuationStream = StreamUtil.getPunctuationStream(stream); declareGrouping(boltDeclarer, parent, punctuationStream, GroupingInfo.all()); } streamToInitialProcessor.put(stream, curNode); } } } } return streamToInitialProcessor; }
private void addSink(TopologyBuilder topologyBuilder, SinkNode sinkNode) { IComponent bolt = sinkNode.getBolt(); BoltDeclarer boltDeclarer; if (bolt instanceof IRichBolt) { boltDeclarer = topologyBuilder.setBolt(sinkNode.getComponentId(), (IRichBolt) bolt, sinkNode.getParallelism()); } else if (bolt instanceof IBasicBolt) { boltDeclarer = topologyBuilder.setBolt(sinkNode.getComponentId(), (IBasicBolt) bolt, sinkNode.getParallelism()); } else { throw new IllegalArgumentException("Expect IRichBolt or IBasicBolt in addBolt"); } for (Node parent : parentNodes(sinkNode)) { for (String stream : sinkNode.getParentStreams(parent)) { declareGrouping(boltDeclarer, parent, stream, nodeGroupingInfo.get(parent, stream)); } } }