private boolean hasStatefulProcessor(List<ProcessorNode> processorNodes) { for (ProcessorNode node : processorNodes) { if (node.getProcessor() instanceof StatefulProcessor) { return true; } } return false; }
private boolean shouldPartitionByKey() { if (node.getParallelism() == 1) { return false; } /* * if the current processor preserves the key and is * already partitioned on key, skip the re-partition. */ if (node instanceof ProcessorNode) { ProcessorNode pn = (ProcessorNode) node; Fields fields = pn.getGroupingInfo() == null ? null : pn.getGroupingInfo().getFields(); if (pn.isPreservesKey() && fields != null && fields.equals(KEY)) { return false; } } return true; }
private ProcessorContext createEmittingContext(ProcessorNode processorNode) { List<EmittingProcessorContext> emittingContexts = new ArrayList<>(); for (String stream : processorNode.getOutputStreams()) { EmittingProcessorContext emittingContext = new EmittingProcessorContext(processorNode, outputCollector, stream); emittingContexts.add(emittingContext); } emittingProcessorContexts.addAll(emittingContexts); outgoingProcessors.add(processorNode); return new ChainedProcessorContext(processorNode, emittingContexts); }
private void setUpProcessorBolt(Processor<?> processor, Set<String> windowedParentStreams, boolean isWindowed, String tsFieldName) { ProcessorNode node = new ProcessorNode(processor, "outputstream", new Fields("value")); node.setWindowedParentStreams(windowedParentStreams); node.setWindowed(isWindowed); Mockito.when(mockStreamToProcessors.get(Mockito.anyString())).thenReturn(Collections.singletonList(node)); Mockito.when(mockStreamToProcessors.keySet()).thenReturn(Collections.singleton("inputstream")); Map<GlobalStreamId, Grouping> mockSources = Mockito.mock(Map.class); GlobalStreamId mockGlobalStreamId = Mockito.mock(GlobalStreamId.class); Mockito.when(mockTopologyContext.getThisSources()).thenReturn(mockSources); Mockito.when(mockSources.keySet()).thenReturn(Collections.singleton(mockGlobalStreamId)); Mockito.when(mockGlobalStreamId.get_streamId()).thenReturn("inputstream"); Mockito.when(mockGlobalStreamId.get_componentId()).thenReturn("bolt0"); Mockito.when(mockTopologyContext.getComponentTasks(Mockito.anyString())).thenReturn(Collections.singletonList(1)); graph.addVertex(node); bolt = new ProcessorBolt("bolt1", graph, Collections.singletonList(node)); if (tsFieldName != null && !tsFieldName.isEmpty()) { bolt.setTimestampField(tsFieldName); } bolt.setStreamToInitialProcessors(mockStreamToProcessors); bolt.prepare(new HashMap<>(), mockTopologyContext, mockOutputCollector); }
private void doProcessCurGroup(TopologyBuilder topologyBuilder, List<ProcessorNode> group) { String boltId = UniqueIdGen.getInstance().getUniqueBoltId(); for (ProcessorNode processorNode : group) { processorNode.setComponentId(boltId); processorNode.setWindowedParentStreams(getWindowedParentStreams(processorNode)); } final Set<ProcessorNode> initialProcessors = initialProcessors(group); Set<Window<?, ?>> windowParams = getWindowParams(initialProcessors); if (windowParams.isEmpty()) { if (hasStatefulProcessor(group)) { addStatefulBolt(topologyBuilder, boltId, initialProcessors, group); } else { addBolt(topologyBuilder, boltId, initialProcessors, group); } } else if (windowParams.size() == 1) { addWindowedBolt(topologyBuilder, boltId, initialProcessors, windowParams.iterator().next(), group); } else { throw new IllegalStateException("More than one window config for current group " + group); } }
private void splitStatefulProcessor(ProcessorNode processorNode, TopologyBuilder topologyBuilder) { for (Node parent : StreamUtil.<Node>getParents(graph, processorNode)) { ProcessorNode identity = new ProcessorNode(new MapProcessor<>(new IdentityFunction<>()), UniqueIdGen.getInstance().getUniqueStreamId(), parent.getOutputFields()); addNode(parent, identity); graph.removeEdge(parent, processorNode); processorNode.removeParentStreams(parent); addNode(identity, processorNode); curGroup.add(identity); } processCurGroup(topologyBuilder); }
private void setUpWindowedProcessorBolt(Processor<?> processor, Window<?, ?> window) { ProcessorNode node = new ProcessorNode(processor, "outputstream", new Fields("value")); node.setWindowed(true); Mockito.when(mockStreamToProcessors.get(Mockito.anyString())).thenReturn(Collections.singletonList(node)); Mockito.when(mockStreamToProcessors.keySet()).thenReturn(Collections.singleton("inputstream")); graph = new DefaultDirectedGraph<>(new StreamsEdgeFactory()); graph.addVertex(node); bolt = new WindowedProcessorBolt("bolt1", graph, Collections.singletonList(node), window); bolt.setStreamToInitialProcessors(mockStreamToProcessors); bolt.prepare(new HashMap<>(), mockTopologyContext, mockOutputCollector); }
private void setUpStatefulProcessorBolt(Processor<?> processor) { ProcessorNode node = new ProcessorNode(processor, "outputstream", new Fields("value")); node.setEmitsPair(true); Mockito.when(mockStreamToProcessors.get(Mockito.anyString())).thenReturn(Collections.singletonList(node)); graph = new DefaultDirectedGraph(new StreamsEdgeFactory()); graph.addVertex(node); bolt = new StatefulProcessorBolt<>("bolt1", graph, Collections.singletonList(node)); bolt.setStreamToInitialProcessors(mockStreamToProcessors); bolt.prepare(new HashMap<>(), mockTopologyContext, mockOutputCollector); bolt.initState(mockKeyValueState); }
ProcessorNode makeProcessorNode(Processor<?> processor, Fields outputFields, boolean preservesKey) { return new ProcessorNode(processor, UniqueIdGen.getInstance().getUniqueStreamId(), outputFields, preservesKey); }
void declareOutputFields(OutputFieldsDeclarer declarer) { for (ProcessorNode node : nodes) { for (String stream : node.getOutputStreams()) { if (timestampField == null) { declarer.declareStream(stream, node.getOutputFields()); } else { List<String> fields = new ArrayList<>(); fields.addAll(node.getOutputFields().toList()); fields.add(timestampField); declarer.declareStream(stream, new Fields(fields)); } /* * Declare a separate 'punctuation' stream per output stream so that the receiving bolt * can subscribe to this stream with 'ALL' grouping and process the punctuation once it * receives from all upstream tasks. */ declarer.declareStream(StreamUtil.getPunctuationStream(stream), StreamUtil.getPunctuationFields()); } } }
Multimap<String, ProcessorNode> streamToChildren = ArrayListMultimap.create(); for (ProcessorNode child : children) { for (String stream : child.getParentStreams(processorNode)) { streamToChildren.put(stream, child); processorNode.initProcessorContext(processorContext);
@Override public Set<String> getWindowedParentStreams() { return processorNode.getWindowedParentStreams(); } }
public EmittingProcessorContext(ProcessorNode processorNode, OutputCollector collector, String outputStreamId) { this.processorNode = processorNode; this.outputStreamId = outputStreamId; this.collector = collector; outputFields = processorNode.getOutputFields(); punctuation = new Values(PUNCTUATION); punctuationStreamId = StreamUtil.getPunctuationStream(outputStreamId); }
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; }
@Override public <T> void forward(T input) { if (PUNCTUATION.equals(input)) { emit(punctuation, punctuationStreamId); maybeAck(); } else if (processorNode.emitsPair()) { Pair<?, ?> value = (Pair<?, ?>) input; emit(new Values(value.getFirst(), value.getSecond()), outputStreamId); } else { emit(new Values(input), outputStreamId); } }
@Override public Set<String> getWindowedParentStreams() { return processorNode.getWindowedParentStreams(); }
private StateQueryProcessor<?, ?> getStateQueryProcessor(List<ProcessorNode> group) { for (ProcessorNode node : group) { if (node.getProcessor() instanceof StateQueryProcessor) { return (StateQueryProcessor<?, ?>) node.getProcessor(); } } return null; }
@Override public Set<String> getWindowedParentStreams() { return processorNode.getWindowedParentStreams(); }
/** * Returns an array of streams by splitting the given stream into multiple branches based on the given predicates. The predicates are * applied in the given order to the values of this stream and the result is forwarded to the corresponding (index based) result stream * based on the (index of) predicate that matches. * <p> * <b>Note:</b> If none of the predicates match a value, that value is dropped. * </p> * * @param predicates the predicates * @return an array of result streams (branches) corresponding to the given predicates */ @SuppressWarnings("unchecked") public Stream<T>[] branch(Predicate<? super T>... predicates) { List<Stream<T>> childStreams = new ArrayList<>(); if (predicates.length > 0) { BranchProcessor<T> branchProcessor = new BranchProcessor<>(); Node branchNode = addProcessorNode(branchProcessor, VALUE); for (Predicate<? super T> predicate : predicates) { // create a child node (identity) per branch ProcessorNode child = makeProcessorNode(new MapProcessor<>(new IdentityFunction<>()), node.getOutputFields()); String branchStream = child.getOutputStreams().iterator().next() + "-branch"; // branchStream is the parent stream that connects branch processor to this child branchNode.addOutputStream(branchStream); addNode(branchNode, child, branchStream); childStreams.add(new Stream<>(streamBuilder, child)); branchProcessor.addPredicate(predicate, branchStream); } } return childStreams.toArray((Stream<T>[]) new Stream[childStreams.size()]); }
@SuppressWarnings("unchecked") private Set<StatefulProcessor<K, V>> getStatefulProcessors(List<ProcessorNode> nodes) { Set<StatefulProcessor<K, V>> statefulProcessors = new HashSet<>(); int updateStateByKeyCount = 0; for (ProcessorNode node : nodes) { if (node.getProcessor() instanceof StatefulProcessor) { statefulProcessors.add((StatefulProcessor<K, V>) node.getProcessor()); if (node.getProcessor() instanceof UpdateStateByKeyProcessor) { if (++updateStateByKeyCount > 1) { throw new IllegalArgumentException("Cannot have more than one updateStateByKey processor " + "in a StatefulProcessorBolt"); } } } } return statefulProcessors; } }