private StreamBolt addWindowedBolt(TopologyBuilder topologyBuilder, String boltId, Set<ProcessorNode> initialProcessors, Window<?, ?> windowParam, List<ProcessorNode> group) { WindowedProcessorBolt bolt = new WindowedProcessorBolt(boltId, graph, group, windowParam); BoltDeclarer boltDeclarer = topologyBuilder.setBolt(boltId, bolt, getParallelism(group)); bolt.setStreamToInitialProcessors(wireBolt(group, boltDeclarer, initialProcessors)); streamBolts.put(bolt, boltDeclarer); return bolt; }
private void setWindowConfig() { if (window instanceof SlidingWindows) { setSlidingWindowParams(window.getWindowLength(), window.getSlidingInterval()); } else if (window instanceof TumblingWindows) { setTumblingWindowParams(window.getWindowLength()); } if (window.getTimestampField() != null) { withTimestampField(window.getTimestampField()); } if (window.getLag() != null) { withLag(window.getLag()); } if (window.getLateTupleStream() != null) { withLateTupleStream(window.getLateTupleStream()); } }
WindowedProcessorBolt(String id, DirectedGraph<Node, Edge> graph, List<ProcessorNode> nodes, Window<?, ?> window) { delegate = new ProcessorBoltDelegate(id, graph, nodes); this.window = window; setWindowConfig(); }
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); }
@Test public void testEmit() throws Exception { Window<?, ?> window = TumblingWindows.of(BaseWindowedBolt.Count.of(2)); setUpWindowedProcessorBolt(new AggregateProcessor<>(new Count<>()), window); bolt.execute(getMockTupleWindow(mockTuple1, mockTuple2, mockTuple3)); ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class); ArgumentCaptor<String> os = ArgumentCaptor.forClass(String.class); Mockito.verify(mockOutputCollector, Mockito.times(2)).emit(os.capture(), values.capture()); assertEquals("outputstream", os.getAllValues().get(0)); assertEquals(new Values(3L), values.getAllValues().get(0)); assertEquals("outputstream__punctuation", os.getAllValues().get(1)); assertEquals(new Values(WindowNode.PUNCTUATION), values.getAllValues().get(1)); }