@Override public StormTopology build() { return streamBuilder.build(); }
@SuppressWarnings("unchecked") public static void main(String[] args) throws Exception { StreamBuilder builder = new StreamBuilder(); Stream<Integer>[] evenAndOdd = builder /* * Create a stream of random numbers from a spout that * emits random integers by extracting the tuple value at index 0. */ .newStream(new RandomIntegerSpout(), new ValueMapper<Integer>(0)) /* * Split the stream of numbers into streams of * even and odd numbers. The first stream contains even * and the second contains odd numbers. */ .branch(x -> (x % 2) == 0, x -> (x % 2) == 1); evenAndOdd[0].forEach(x -> LOG.info("EVEN> " + x)); evenAndOdd[1].forEach(x -> LOG.info("ODD > " + x)); Config config = new Config(); String topoName = "branchExample"; if (args.length > 0) { topoName = args[0]; } config.setNumWorkers(1); StormSubmitter.submitTopologyWithProgressBar(topoName, config, builder.build()); }
@Test(expected = IllegalArgumentException.class) public void testSpoutNoDefaultStream() throws Exception { Stream<Tuple> stream = streamBuilder.newStream(newSpout("test")); stream.filter(x -> true); streamBuilder.build(); }
@Test public void testSpoutToBolt() throws Exception { Stream<Tuple> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID)); stream.to(newBolt()); StormTopology topology = streamBuilder.build(); assertEquals(1, topology.get_spouts_size()); assertEquals(1, topology.get_bolts_size()); String spoutId = topology.get_spouts().keySet().iterator().next(); Map<GlobalStreamId, Grouping> expected = new HashMap<>(); expected.put(new GlobalStreamId(spoutId, "default"), Grouping.shuffle(new NullStruct())); assertEquals(expected, topology.get_bolts().values().iterator().next().get_common().get_inputs()); }
@Test public void testGroupBy() throws Exception { PairStream<String, String> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new PairValueMapper<>(0, 1), 2); stream.window(TumblingWindows.of(BaseWindowedBolt.Count.of(10))).aggregateByKey(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"); assertEquals(Grouping.shuffle(new NullStruct()), bolt1.get_common().get_inputs().values().iterator().next()); assertEquals(Grouping.fields(Collections.singletonList("key")), bolt2.get_common().get_inputs().values().iterator().next()); }
@Test public void testBranch() throws Exception { Stream<Tuple> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID)); Stream<Tuple>[] streams = stream.branch(x -> true); StormTopology topology = streamBuilder.build(); assertEquals(1, topology.get_spouts_size()); assertEquals(1, topology.get_bolts_size()); Map<GlobalStreamId, Grouping> expected = new HashMap<>(); String spoutId = topology.get_spouts().keySet().iterator().next(); expected.put(new GlobalStreamId(spoutId, "default"), Grouping.shuffle(new NullStruct())); assertEquals(expected, topology.get_bolts().values().iterator().next().get_common().get_inputs()); assertEquals(1, streams.length); assertEquals(1, streams[0].node.getOutputStreams().size()); String parentStream = streams[0].node.getOutputStreams().iterator().next() + "-branch"; assertEquals(1, streams[0].node.getParents(parentStream).size()); Node processorNdoe = streams[0].node.getParents(parentStream).iterator().next(); assertTrue(processorNdoe instanceof ProcessorNode); assertTrue(((ProcessorNode) processorNdoe).getProcessor() instanceof BranchProcessor); assertTrue(processorNdoe.getParents("default").iterator().next() instanceof SpoutNode); }
@Test public void testPartitionByKeySinglePartition() { TopologyContext mockContext = Mockito.mock(TopologyContext.class); OutputCollector mockCollector = Mockito.mock(OutputCollector.class); Stream<Integer> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0)); stream.mapToPair(x -> Pair.of(x, x)) .reduceByKey((x, y) -> x + y) .print(); StormTopology topology = streamBuilder.build(); assertEquals(1, topology.get_bolts_size()); } }
@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()); }
StormSubmitter.submitTopologyWithProgressBar(topoName, config, builder.build());
public static void main(String[] args) throws Exception { StreamBuilder builder = new StreamBuilder(); // a stream of (number, square) pairs PairStream<Integer, Integer> squares = builder .newStream(new NumberSpout(x -> x * x), new PairValueMapper<>(0, 1)); // a stream of (number, cube) pairs PairStream<Integer, Integer> cubes = builder .newStream(new NumberSpout(x -> x * x * x), new PairValueMapper<>(0, 1)); // create a windowed stream of five seconds duration squares.window(TumblingWindows.of(Duration.seconds(5))) /* * Join the squares and the cubes stream within the window. * The values in the squares stream having the same key as that * of the cubes stream within the window will be joined together. */ .join(cubes) /** * The results should be of the form (number, (square, cube)) */ .print(); Config config = new Config(); String topoName = JoinExample.class.getName(); if (args.length > 0) { topoName = args[0]; } config.setNumWorkers(1); StormSubmitter.submitTopologyWithProgressBar(topoName, config, builder.build()); }
@Test public void testMultiPartitionByKeyWithRepartition() { TopologyContext mockContext = Mockito.mock(TopologyContext.class); OutputCollector mockCollector = Mockito.mock(OutputCollector.class); Map<GlobalStreamId, Grouping> expected = new HashMap<>(); expected.put(new GlobalStreamId("bolt2", "s3"), Grouping.fields(Collections.singletonList("key"))); expected.put(new GlobalStreamId("bolt2", "s3__punctuation"), Grouping.all(new NullStruct())); Stream<Integer> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0)); stream.mapToPair(x -> Pair.of(x, x)) .window(TumblingWindows.of(BaseWindowedBolt.Count.of(10))) .reduceByKey((x, y) -> x + y) .repartition(10) .reduceByKey((x, y) -> 0) .print(); StormTopology topology = streamBuilder.build(); assertEquals(3, topology.get_bolts_size()); assertEquals(expected, topology.get_bolts().get("bolt3").get_common().get_inputs()); }
@Test public void testBranchAndJoin() throws Exception { TopologyContext mockContext = Mockito.mock(TopologyContext.class); OutputCollector mockCollector = Mockito.mock(OutputCollector.class); Stream<Integer> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0), 2); Stream<Integer>[] streams = stream.branch(x -> x % 2 == 0, x -> x % 2 == 1); PairStream<Integer, Pair<Integer, Integer>> joined = streams[0].mapToPair(x -> Pair.of(x, 1)).join(streams[1].mapToPair(x -> Pair.of(x, 1))); assertTrue(joined.getNode() instanceof ProcessorNode); StormTopology topology = streamBuilder.build(); assertEquals(2, topology.get_bolts_size()); }
public static void main(String[] args) throws Exception { StreamBuilder builder = new StreamBuilder(); /** * The spout emits sequences of (Integer, Long, Long). TupleValueMapper can be used to extract fields * from the values and produce a stream of typed tuple (Tuple3<Integer, Long, Long> in this case. */ Stream<Tuple3<Integer, Long, Long>> stream = builder.newStream(new RandomIntegerSpout(), TupleValueMappers.of(0, 1, 2)); PairStream<Long, Integer> pairs = stream.mapToPair(t -> Pair.of(t._2 / 10000, t._1)); pairs.window(TumblingWindows.of(Count.of(10))).groupByKey().print(); String topoName = "test"; if (args.length > 0) { topoName = args[0]; } Config config = new Config(); config.setNumWorkers(1); StormSubmitter.submitTopologyWithProgressBar(topoName, config, builder.build()); } }
@Test public void testMultiPartitionByKey() { TopologyContext mockContext = Mockito.mock(TopologyContext.class); OutputCollector mockCollector = Mockito.mock(OutputCollector.class); Stream<Integer> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0)); stream.mapToPair(x -> Pair.of(x, x)) .window(TumblingWindows.of(BaseWindowedBolt.Count.of(10))) .reduceByKey((x, y) -> x + y) .reduceByKey((x, y) -> 0) .print(); StormTopology topology = streamBuilder.build(); assertEquals(2, topology.get_bolts_size()); }
@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()); }
StormSubmitter.submitTopologyWithProgressBar(topoName, config, builder.build());
StormSubmitter.submitTopologyWithProgressBar(topoName, config, builder.build());
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()); }
StormSubmitter.submitTopologyWithProgressBar(topoName, config, builder.build());