/** * Counts the number of values in this stream. This does a global count of values across all partitions. * <p> * If the stream is windowed, the counts are emitted after each window activation and represents the count of elements that fall within * that window. If the stream is not windowed, the count is emitted as each new element in the stream is processed. * </p> * * @return the new stream */ public Stream<Long> count() { return aggregate(new Count<>()); }
@Test public void testGlobalAggregate() throws Exception { Stream<String> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0), 2); stream.aggregate(new Count<>()); StormTopology topology = streamBuilder.build(); assertEquals(2, topology.get_bolts_size()); Bolt bolt1 = topology.get_bolts().get("bolt1"); Bolt bolt2 = topology.get_bolts().get("bolt2"); String spoutId = topology.get_spouts().keySet().iterator().next(); Map<GlobalStreamId, Grouping> expected1 = new HashMap<>(); expected1.put(new GlobalStreamId(spoutId, "default"), Grouping.shuffle(new NullStruct())); Map<GlobalStreamId, Grouping> expected2 = new HashMap<>(); expected2.put(new GlobalStreamId("bolt1", "s1"), Grouping.fields(Collections.emptyList())); expected2.put(new GlobalStreamId("bolt1", "s1__punctuation"), Grouping.all(new NullStruct())); assertEquals(expected1, bolt1.get_common().get_inputs()); assertEquals(expected2, bolt2.get_common().get_inputs()); }
@SuppressWarnings("unchecked") public static void main(String[] args) throws Exception { StreamBuilder builder = new StreamBuilder(); /** * Computes average of the stream of numbers emitted by the spout. Internally the per-partition * sum and counts are accumulated and emitted to a downstream task where the partially accumulated * results are merged and the final result is emitted. */ builder.newStream(new RandomIntegerSpout(), new ValueMapper<Integer>(0), 2) .window(TumblingWindows.of(BaseWindowedBolt.Duration.seconds(5))) .filter(x -> x > 0 && x < 500) .aggregate(new Avg()) .print(); Config config = new Config(); String topoName = "AGG_EXAMPLE"; if (args.length > 0) { topoName = args[0]; } config.setNumWorkers(1); StormSubmitter.submitTopologyWithProgressBar(topoName, config, builder.build()); }
@Test public void testRepartition() throws Exception { Stream<String> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0)); stream.repartition(3).filter(x -> true).repartition(2).filter(x -> true).aggregate(new Count<>()); StormTopology topology = streamBuilder.build(); assertEquals(1, topology.get_spouts_size()); SpoutSpec spout = topology.get_spouts().get("spout1"); assertEquals(4, topology.get_bolts_size()); Bolt bolt1 = topology.get_bolts().get("bolt1"); Bolt bolt2 = topology.get_bolts().get("bolt2"); Bolt bolt3 = topology.get_bolts().get("bolt3"); Bolt bolt4 = topology.get_bolts().get("bolt4"); assertEquals(1, spout.get_common().get_parallelism_hint()); assertEquals(1, bolt1.get_common().get_parallelism_hint()); assertEquals(3, bolt2.get_common().get_parallelism_hint()); assertEquals(2, bolt3.get_common().get_parallelism_hint()); assertEquals(2, bolt4.get_common().get_parallelism_hint()); }