/** * {@inheritDoc} */ public PairStream<K, V> filter(Predicate<? super Pair<K, V>> predicate) { return toPairStream(super.filter(predicate)); }
@Test(expected = IllegalArgumentException.class) public void testSpoutNoDefaultStream() throws Exception { Stream<Tuple> stream = streamBuilder.newStream(newSpout("test")); stream.filter(x -> true); streamBuilder.build(); }
@Override public void streamsPlan(StreamsPlanCreator planCreator) throws Exception { // SingleRel RelNode input = getInput(); StormRelUtils.getStormRelInput(input).streamsPlan(planCreator); Stream<Values> inputStream = planCreator.pop(); List<RexNode> childExps = getChildExps(); RelDataType inputRowType = getInput(0).getRowType(); String filterClassName = StormRelUtils.getClassName(this); ExecutableExpression filterInstance = planCreator.createScalarInstance(childExps, inputRowType, filterClassName); EvaluationFilter evalFilter = new EvaluationFilter(filterInstance, planCreator.getDataContext()); final Stream<Values> finalStream = inputStream.filter(evalFilter); planCreator.addStream(finalStream); } }
@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()); }