/** * Join the values of this stream with the values having the same key from the other stream. * <p> * Note: The parallelism of this stream is carried forward to the joined stream. * </p> * * @param otherStream the other stream * @param <V1> the type of the values in the other stream * @return the new stream */ public <V1> PairStream<K, Pair<V, V1>> join(PairStream<K, V1> otherStream) { return join(otherStream, new PairValueJoiner<>()); }
/** * Returns a new stream where the values that arrive within a window having the same key will be reduced by repeatedly applying the * reducer. * * @param reducer the reducer * @param window the window configuration * @return the new stream */ public PairStream<K, V> reduceByKeyAndWindow(Reducer<V> reducer, Window<?, ?> window) { return partitionByKey().window(window).reduceByKey(reducer); }
/** * Counts the values for each key of this stream. * * @return the new stream */ public PairStream<K, Long> countByKey() { return aggregateByKey(new Count<>()); }
/** * Returns a new stream where the values are grouped by keys and the given window. The values that arrive within a window having the * same key will be merged together and returned as an Iterable of values mapped to the key. * * @param window the window configuration * @return the new stream */ public PairStream<K, Iterable<V>> groupByKeyAndWindow(Window<?, ?> window) { return partitionByKey().window(window).aggregatePartition(new MergeValues<>()); }
private <R, V1> PairStream<K, R> coGroupByKeyPartition(PairStream<K, V1> otherStream) { String firstStream = stream; String secondStream = otherStream.stream; Node coGroupNode = addProcessorNode( new CoGroupByKeyProcessor<>(firstStream, secondStream), KEY_VALUE, true); addNode(otherStream.getNode(), coGroupNode, coGroupNode.getParallelism()); return new PairStream<>(streamBuilder, coGroupNode); }
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 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()); }
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()); } }
/** * Returns a new stream by applying a {@link Function} to the value of each key-value pairs in this stream. * * @param function the mapping function * @param <R> the result type * @return the new stream */ public <R> PairStream<K, R> mapValues(Function<? super V, ? extends R> function) { return new PairStream<>(streamBuilder, addProcessorNode(new MapValuesProcessor<>(function), KEY_VALUE, true)); }
.countByKey() .updateStateByKey(0L, (state, count) -> state + count) .print();
@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 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()); }
.groupByKeyAndWindow(SlidingWindows.of(Count.of(6), Count.of(3))) .print(); .reduceByKeyAndWindow((x, y) -> x > y ? x : y, SlidingWindows.of(Count.of(6), Count.of(3))) .print();
.countByKey() .filter(x -> x.getSecond() >= 5) .print();
@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()); } }
.updateStateByKey(0L, (count, val) -> count + 1); .stateQuery(ss).print();
.countByKey() .to(redisStoreBolt);
/** * Update the state by applying the given state update function to the previous state of the key and the new value for the key. This * internally uses {@link org.apache.storm.topology.IStatefulBolt} to save the state. Use {@link Config#TOPOLOGY_STATE_PROVIDER} to * choose the state implementation. * * @param stateUpdateFn the state update function * @param <R> the result type * @return the {@link StreamState} which can be used to query the state */ public <R> StreamState<K, R> updateStateByKey(R initialValue, BiFunction<? super R, ? super V, ? extends R> stateUpdateFn) { return updateStateByKey(StateUpdater.of(initialValue, stateUpdateFn)); }
@Override public void streamsPlan(StreamsPlanCreator planCreator) throws Exception { // SingleRel RelNode input = getInput(); StormRelUtils.getStormRelInput(input).streamsPlan(planCreator); Stream<Values> inputStream = planCreator.pop(); Preconditions.checkArgument(isInsert(), "Only INSERT statement is supported."); // Calcite ensures that the value is structurized to the table definition // hence we can use PK index directly // To elaborate, if table BAR is defined as ID INTEGER PK, NAME VARCHAR, DEPTID INTEGER // and query like INSERT INTO BAR SELECT NAME, ID FROM FOO is executed, // Calcite makes the projection ($1 <- ID, $0 <- NAME, null) to the value before INSERT. String tableName = Joiner.on('.').join(getTable().getQualifiedName()); IRichBolt consumer = planCreator.getSources().get(tableName).getConsumer(); // To make logic simple, it assumes that all the tables have one PK (which it should be extended to support composed key), // and provides PairStream(KeyedStream) to consumer bolt. inputStream.mapToPair(new StreamInsertMapToPairFunction(primaryKeyIndex)).to(consumer); planCreator.addStream(inputStream); }