From e251573d524f6a6a11c7821ba93fd155d9beb770 Mon Sep 17 00:00:00 2001
From: Arun Mahadevan
Date: Mon, 19 Sep 2016 23:50:16 +0530
Subject: [PATCH 01/29] [STORM-1961] Stream api for storm core use cases
The initial version of unified stream api for expressing streaming computation pipelines over the storm core spouts and bolts.
Right now this provides at-least once guarantees and addresses only the storm core use cases.
For high level design see - https://issues.apache.org/jira/secure/attachment/12827547/UnifiedStreamapiforStorm.pdf
A few examples have been added which should give a basic idea on how to use the apis. More examples and detailed documentation will be added as followup tasks.
---
.../storm/starter/streams/BranchExample.java | 72 +++
.../streams/GroupByKeyAndWindowExample.java | 109 ++++
.../storm/starter/streams/JoinExample.java | 110 ++++
.../starter/streams/StateQueryExample.java | 112 ++++
.../starter/streams/StatefulWordCount.java | 85 +++
.../starter/streams/WindowedWordCount.java | 90 +++
.../starter/streams/WordCountToBolt.java | 111 ++++
.../jvm/org/apache/storm/streams/Edge.java | 41 ++
.../apache/storm/streams/GroupingInfo.java | 84 +++
.../jvm/org/apache/storm/streams/Node.java | 129 ++++
.../jvm/org/apache/storm/streams/Pair.java | 78 +++
.../org/apache/storm/streams/PairStream.java | 241 +++++++
.../apache/storm/streams/PartitionNode.java | 42 ++
.../apache/storm/streams/ProcessorBolt.java | 70 +++
.../storm/streams/ProcessorBoltDelegate.java | 285 +++++++++
.../apache/storm/streams/ProcessorNode.java | 81 +++
.../apache/storm/streams/RefCountedTuple.java | 64 ++
.../org/apache/storm/streams/SinkNode.java | 44 ++
.../org/apache/storm/streams/SpoutNode.java | 48 ++
.../storm/streams/StatefulProcessorBolt.java | 116 ++++
.../jvm/org/apache/storm/streams/Stream.java | 393 ++++++++++++
.../org/apache/storm/streams/StreamBolt.java | 38 ++
.../apache/storm/streams/StreamBuilder.java | 591 ++++++++++++++++++
.../org/apache/storm/streams/StreamState.java | 43 ++
.../org/apache/storm/streams/StreamUtil.java | 62 ++
.../storm/streams/StreamsEdgeFactory.java | 29 +
.../jvm/org/apache/storm/streams/Tuple3.java | 49 ++
.../org/apache/storm/streams/UniqueIdGen.java | 56 ++
.../org/apache/storm/streams/WindowNode.java | 38 ++
.../storm/streams/WindowedProcessorBolt.java | 137 ++++
.../storm/streams/operations/Aggregator.java | 42 ++
.../storm/streams/operations/Consumer.java | 32 +
.../streams/operations/FlatMapFunction.java | 27 +
.../storm/streams/operations/Function.java | 34 +
.../streams/operations/IdentityFunction.java | 31 +
.../storm/streams/operations/Operation.java | 26 +
.../operations/PairFlatMapFunction.java | 30 +
.../streams/operations/PairFunction.java | 30 +
.../streams/operations/PairValueJoiner.java | 40 ++
.../storm/streams/operations/Predicate.java | 33 +
.../streams/operations/PrintConsumer.java | 30 +
.../storm/streams/operations/Reducer.java | 35 ++
.../storm/streams/operations/ValueJoiner.java | 36 ++
.../streams/operations/aggregators/Count.java | 37 ++
.../streams/operations/aggregators/Sum.java | 35 ++
.../operations/mappers/PairValueMapper.java | 51 ++
.../operations/mappers/TupleValueMapper.java | 30 +
.../operations/mappers/ValueMapper.java | 45 ++
.../operations/mappers/ValuesMapper.java | 48 ++
.../processors/AggregateByKeyProcessor.java | 54 ++
.../processors/AggregateProcessor.java | 45 ++
.../streams/processors/BaseProcessor.java | 107 ++++
.../streams/processors/BatchProcessor.java | 25 +
.../streams/processors/BranchProcessor.java | 41 ++
.../processors/ChainedProcessorContext.java | 66 ++
.../processors/EmittingProcessorContext.java | 170 +++++
.../streams/processors/FilterProcessor.java | 35 ++
.../streams/processors/FlatMapProcessor.java | 35 ++
.../processors/FlatMapValuesProcessor.java | 36 ++
.../streams/processors/ForEachProcessor.java | 33 +
.../ForwardingProcessorContext.java | 102 +++
.../streams/processors/JoinProcessor.java | 112 ++++
.../streams/processors/MapProcessor.java | 33 +
.../processors/MapValuesProcessor.java | 34 +
.../streams/processors/PeekProcessor.java | 34 +
.../storm/streams/processors/Processor.java | 51 ++
.../streams/processors/ProcessorContext.java | 59 ++
.../processors/ReduceByKeyProcessor.java | 52 ++
.../streams/processors/ReduceProcessor.java | 41 ++
.../processors/StateQueryProcessor.java | 48 ++
.../streams/processors/StatefulProcessor.java | 36 ++
.../processors/UpdateStateByKeyProcessor.java | 49 ++
.../storm/streams/windowing/BaseWindow.java | 64 ++
.../streams/windowing/SlidingWindows.java | 151 +++++
.../streams/windowing/TumblingWindows.java | 119 ++++
.../storm/streams/windowing/Window.java | 70 +++
.../StatefulWindowedBoltExecutor.java | 4 +-
.../storm/topology/WindowedBoltExecutor.java | 4 +-
.../storm/topology/base/BaseWindowedBolt.java | 37 +-
.../AbstractTridentWindowManager.java | 2 +-
.../storm/windowing/CountEvictionPolicy.java | 8 +-
.../storm/windowing/EvictionPolicy.java | 7 +
.../storm/windowing/TimeEvictionPolicy.java | 17 +-
.../storm/windowing/TupleWindowImpl.java | 11 +
.../WatermarkCountEvictionPolicy.java | 10 +-
.../WatermarkTimeEvictionPolicy.java | 4 +-
.../org/apache/storm/windowing/Window.java | 7 +
.../windowing/WindowLifecycleListener.java | 3 +-
.../apache/storm/windowing/WindowManager.java | 2 +-
.../storm/streams/ProcessorBoltTest.java | 165 +++++
.../streams/StatefulProcessorBoltTest.java | 100 +++
.../storm/streams/StreamBuilderTest.java | 219 +++++++
.../streams/WindowedProcessorBoltTest.java | 110 ++++
.../storm/windowing/WindowManagerTest.java | 2 +-
94 files changed, 6505 insertions(+), 29 deletions(-)
create mode 100644 examples/storm-starter/src/jvm/org/apache/storm/starter/streams/BranchExample.java
create mode 100644 examples/storm-starter/src/jvm/org/apache/storm/starter/streams/GroupByKeyAndWindowExample.java
create mode 100644 examples/storm-starter/src/jvm/org/apache/storm/starter/streams/JoinExample.java
create mode 100644 examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java
create mode 100644 examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java
create mode 100644 examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java
create mode 100644 examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/Edge.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/GroupingInfo.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/Node.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/Pair.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/PairStream.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/PartitionNode.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/ProcessorBolt.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/ProcessorNode.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/RefCountedTuple.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/SinkNode.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/SpoutNode.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/StatefulProcessorBolt.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/Stream.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/StreamBolt.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/StreamState.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/StreamUtil.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/StreamsEdgeFactory.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/Tuple3.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/UniqueIdGen.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/WindowNode.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/WindowedProcessorBolt.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/Aggregator.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/Consumer.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/FlatMapFunction.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/Function.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/IdentityFunction.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/Operation.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/PairFlatMapFunction.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/PairFunction.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/PairValueJoiner.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/Predicate.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/PrintConsumer.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/Reducer.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/ValueJoiner.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Count.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Sum.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/mappers/PairValueMapper.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/mappers/TupleValueMapper.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/mappers/ValueMapper.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/mappers/ValuesMapper.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/BaseProcessor.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/BatchProcessor.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/BranchProcessor.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/ChainedProcessorContext.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/EmittingProcessorContext.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/FilterProcessor.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/FlatMapProcessor.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/FlatMapValuesProcessor.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/ForEachProcessor.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/ForwardingProcessorContext.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/JoinProcessor.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/MapProcessor.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/MapValuesProcessor.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/PeekProcessor.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/ReduceByKeyProcessor.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/StateQueryProcessor.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/StatefulProcessor.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/UpdateStateByKeyProcessor.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/windowing/BaseWindow.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/windowing/SlidingWindows.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/windowing/TumblingWindows.java
create mode 100644 storm-core/src/jvm/org/apache/storm/streams/windowing/Window.java
create mode 100644 storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java
create mode 100644 storm-core/test/jvm/org/apache/storm/streams/StatefulProcessorBoltTest.java
create mode 100644 storm-core/test/jvm/org/apache/storm/streams/StreamBuilderTest.java
create mode 100644 storm-core/test/jvm/org/apache/storm/streams/WindowedProcessorBoltTest.java
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/BranchExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/BranchExample.java
new file mode 100644
index 00000000000..f5400a5cbbc
--- /dev/null
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/BranchExample.java
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.starter.streams;
+
+import org.apache.storm.Config;
+import org.apache.storm.LocalCluster;
+import org.apache.storm.StormSubmitter;
+import org.apache.storm.starter.spout.RandomIntegerSpout;
+import org.apache.storm.streams.Stream;
+import org.apache.storm.streams.StreamBuilder;
+import org.apache.storm.streams.operations.Predicate;
+import org.apache.storm.streams.operations.mappers.ValueMapper;
+import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An example that demonstrates the usage of {@link Stream#branch(Predicate[])} to split a stream
+ * into multiple branches based on predicates.
+ */
+public class BranchExample {
+ private static final Logger LOG = LoggerFactory.getLogger(BranchExample.class);
+
+ @SuppressWarnings("unchecked")
+ public static void main(String[] args) throws Exception {
+ StreamBuilder builder = new StreamBuilder();
+ Stream[] 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(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();
+ if (args.length > 0) {
+ config.setNumWorkers(1);
+ StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build());
+ } else {
+ LocalCluster cluster = new LocalCluster();
+ cluster.submitTopology("test", config, builder.build());
+ Utils.sleep(60000);
+ cluster.killTopology("test");
+ cluster.shutdown();
+ }
+ }
+
+}
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/GroupByKeyAndWindowExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/GroupByKeyAndWindowExample.java
new file mode 100644
index 00000000000..6b505bdc95e
--- /dev/null
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/GroupByKeyAndWindowExample.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.starter.streams;
+
+import org.apache.storm.Config;
+import org.apache.storm.LocalCluster;
+import org.apache.storm.StormSubmitter;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.streams.PairStream;
+import org.apache.storm.streams.StreamBuilder;
+import org.apache.storm.streams.operations.Reducer;
+import org.apache.storm.streams.operations.mappers.PairValueMapper;
+import org.apache.storm.streams.windowing.SlidingWindows;
+import org.apache.storm.streams.windowing.Window;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseRichSpout;
+import org.apache.storm.topology.base.BaseWindowedBolt.Count;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.Utils;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * An example that shows the usage of {@link PairStream#groupByKeyAndWindow(Window)}
+ * and {@link PairStream#reduceByKeyAndWindow(Reducer, Window)}
+ */
+public class GroupByKeyAndWindowExample {
+ public static void main(String[] args) throws Exception {
+ StreamBuilder builder = new StreamBuilder();
+
+ // a stream of stock quotes
+ builder.newStream(new StockQuotes(), new PairValueMapper(0, 1))
+ /*
+ * The elements having the same key within the window will be grouped
+ * together and the corresponding values will be merged.
+ */
+ .groupByKeyAndWindow(SlidingWindows.of(Count.of(6), Count.of(3)))
+ .print();
+
+ // a stream of stock quotes
+ builder.newStream(new StockQuotes(), new PairValueMapper(0, 1))
+ /*
+ * The elements having the same key within the window will be grouped
+ * together and their values will be reduced using the given reduce function.
+ */
+ .reduceByKeyAndWindow((x, y) -> (x + y) / 2.0, SlidingWindows.of(Count.of(6), Count.of(3)))
+ .print();
+
+ Config config = new Config();
+ if (args.length > 0) {
+ config.setNumWorkers(1);
+ StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build());
+ } else {
+ LocalCluster cluster = new LocalCluster();
+ config.put(Config.TOPOLOGY_STATE_PROVIDER, "org.apache.storm.redis.state.RedisKeyValueStateProvider");
+ cluster.submitTopology("test", config, builder.build());
+ Utils.sleep(60000);
+ cluster.killTopology("test");
+ cluster.shutdown();
+ }
+ }
+
+ private static class StockQuotes extends BaseRichSpout {
+ private final List> values = Arrays.asList(
+ Arrays.asList(new Values("AAPL", 100.0), new Values("GOOG", 780.0), new Values("FB", 125.0)),
+ Arrays.asList(new Values("AAPL", 105.0), new Values("GOOG", 790.0), new Values("FB", 130.0))
+ );
+ private SpoutOutputCollector collector;
+ private int index = 0;
+
+ @Override
+ public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+ this.collector = collector;
+ }
+
+ @Override
+ public void nextTuple() {
+ Utils.sleep(5000);
+ for (Values v : values.get(index)) {
+ collector.emit(v);
+ }
+ index = (index + 1) % values.size();
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ declarer.declare(new Fields("key", "val"));
+ }
+ }
+}
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/JoinExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/JoinExample.java
new file mode 100644
index 00000000000..0b15615f2bb
--- /dev/null
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/JoinExample.java
@@ -0,0 +1,110 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.starter.streams;
+
+import org.apache.storm.Config;
+import org.apache.storm.LocalCluster;
+import org.apache.storm.StormSubmitter;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.streams.PairStream;
+import org.apache.storm.streams.StreamBuilder;
+import org.apache.storm.streams.operations.Function;
+import org.apache.storm.streams.operations.mappers.PairValueMapper;
+import org.apache.storm.streams.windowing.TumblingWindows;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseRichSpout;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.Utils;
+
+import java.util.Map;
+
+import static org.apache.storm.topology.base.BaseWindowedBolt.Duration;
+
+/**
+ * An example that demonstrates the usage of {@link PairStream#join(PairStream)} to join
+ * multiple streams.
+ */
+public class JoinExample {
+ public static void main(String[] args) throws Exception {
+ StreamBuilder builder = new StreamBuilder();
+ // a stream of (number, square) pairs
+ PairStream squares = builder
+ .newStream(new NumberSpout(x -> x * x),
+ new PairValueMapper<>(0, 1));
+ // a stream of (number, cube) pairs
+ PairStream 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();
+ if (args.length > 0) {
+ config.setNumWorkers(1);
+ StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build());
+ } else {
+ LocalCluster cluster = new LocalCluster();
+ cluster.submitTopology("test", config, builder.build());
+ Utils.sleep(60000);
+ cluster.killTopology("test");
+ cluster.shutdown();
+ }
+
+ }
+
+ private static class NumberSpout extends BaseRichSpout {
+ private final Function function;
+ private SpoutOutputCollector collector;
+ private int i = 1;
+
+ NumberSpout(Function function) {
+ this.function = function;
+ }
+
+ @Override
+ public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+ this.collector = collector;
+ }
+
+ @Override
+ public void nextTuple() {
+ Utils.sleep(990);
+ collector.emit(new Values(i, function.apply(i)));
+ i++;
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ declarer.declare(new Fields("key", "val"));
+ }
+ }
+
+}
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java
new file mode 100644
index 00000000000..e76dd3c0fc6
--- /dev/null
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.starter.streams;
+
+import org.apache.storm.Config;
+import org.apache.storm.LocalCluster;
+import org.apache.storm.StormSubmitter;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.streams.Pair;
+import org.apache.storm.streams.Stream;
+import org.apache.storm.streams.StreamBuilder;
+import org.apache.storm.streams.StreamState;
+import org.apache.storm.streams.operations.aggregators.Count;
+import org.apache.storm.streams.operations.mappers.ValueMapper;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.testing.TestWordSpout;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseRichSpout;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.Utils;
+
+import java.util.Map;
+
+/**
+ * An example that uses {@link Stream#stateQuery(StreamState)} to query the state
+ *
+ * You should start a local redis instance before running the 'storm jar' command. By default
+ * the connection will be attempted at localhost:6379. The default
+ * RedisKeyValueStateProvider parameters can be overridden in conf/storm.yaml, for e.g.
+ *
+ */
+public class StateQueryExample {
+ public static void main(String[] args) throws Exception {
+ StreamBuilder builder = new StreamBuilder();
+ StreamState ss = builder.newStream(new TestWordSpout(), new ValueMapper(0))
+ .mapToPair(w -> Pair.of(w, 1))
+ .groupByKey()
+ .updateStateByKey(new Count<>());
+
+ /*
+ * A stream of words emitted by the QuerySpout is used as
+ * the keys to query the state.
+ */
+ builder.newStream(new QuerySpout(), new ValueMapper(0))
+ /*
+ * Queries the state and emits the
+ * matching (key, value) as results. The stream state returned
+ * by the updateStateByKey is passed as the argument to stateQuery.
+ */
+ .stateQuery(ss).print();
+
+ Config config = new Config();
+ // use redis based state store for persistence
+ config.put(Config.TOPOLOGY_STATE_PROVIDER, "org.apache.storm.redis.state.RedisKeyValueStateProvider");
+
+ if (args.length > 0) {
+ config.setNumWorkers(1);
+ StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build());
+ } else {
+ LocalCluster cluster = new LocalCluster();
+ cluster.submitTopology("test", config, builder.build());
+ Utils.sleep(60000);
+ cluster.killTopology("test");
+ cluster.shutdown();
+ }
+ }
+
+ private static class QuerySpout extends BaseRichSpout {
+ private SpoutOutputCollector collector;
+ private final String[] words = {"nathan", "mike"};
+
+ @Override
+ public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+ this.collector = collector;
+ }
+
+ @Override
+ public void nextTuple() {
+ Utils.sleep(2000);
+ for (String word : words) {
+ collector.emit(new Values(word));
+ }
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ declarer.declare(new Fields("word"));
+ }
+ }
+}
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java
new file mode 100644
index 00000000000..f6ae6b07496
--- /dev/null
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java
@@ -0,0 +1,85 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.starter.streams;
+
+import org.apache.storm.Config;
+import org.apache.storm.LocalCluster;
+import org.apache.storm.StormSubmitter;
+import org.apache.storm.streams.Pair;
+import org.apache.storm.streams.PairStream;
+import org.apache.storm.streams.StreamBuilder;
+import org.apache.storm.streams.operations.Aggregator;
+import org.apache.storm.streams.operations.aggregators.Count;
+import org.apache.storm.streams.operations.mappers.ValueMapper;
+import org.apache.storm.testing.TestWordSpout;
+import org.apache.storm.utils.Utils;
+
+/**
+ * A stateful word count that uses {@link PairStream#updateStateByKey(Aggregator)} to
+ * save the counts in a key value state. This example uses Redis state store.
+ *
+ * You should start a local redis instance before running the 'storm jar' command. By default
+ * the connection will be attempted at localhost:6379. The default
+ * RedisKeyValueStateProvider parameters can be overridden in conf/storm.yaml, for e.g.
+ *
+ */
+public class StatefulWordCount {
+ public static void main(String[] args) throws Exception {
+ StreamBuilder builder = new StreamBuilder();
+ // a stream of words
+ builder.newStream(new TestWordSpout(), new ValueMapper(0))
+ /*
+ * create a stream of (word, 1) pairs
+ */
+ .mapToPair(w -> Pair.of(w, 1))
+ /*
+ * group by the word
+ */
+ .groupByKey()
+ /*
+ * update the word counts in the state
+ */
+ .updateStateByKey(new Count<>())
+ /*
+ * convert the state back to a stream and print the results
+ */
+ .toPairStream()
+ .print();
+
+ Config config = new Config();
+ // use redis based state store for persistence
+ config.put(Config.TOPOLOGY_STATE_PROVIDER, "org.apache.storm.redis.state.RedisKeyValueStateProvider");
+
+ if (args.length > 0) {
+ config.setNumWorkers(1);
+ StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build());
+ } else {
+ LocalCluster cluster = new LocalCluster();
+ cluster.submitTopology("test", config, builder.build());
+ Utils.sleep(60000);
+ cluster.killTopology("test");
+ cluster.shutdown();
+ }
+ }
+}
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java
new file mode 100644
index 00000000000..c6e2f4a1491
--- /dev/null
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.starter.streams;
+
+import org.apache.storm.Config;
+import org.apache.storm.LocalCluster;
+import org.apache.storm.StormSubmitter;
+import org.apache.storm.starter.spout.RandomSentenceSpout;
+import org.apache.storm.streams.Pair;
+import org.apache.storm.streams.StreamBuilder;
+import org.apache.storm.streams.operations.aggregators.Count;
+import org.apache.storm.streams.operations.mappers.ValueMapper;
+import org.apache.storm.streams.windowing.TumblingWindows;
+import org.apache.storm.utils.Utils;
+
+import java.util.Arrays;
+
+import static org.apache.storm.topology.base.BaseWindowedBolt.Duration;
+
+/**
+ * A windowed word count example
+ */
+public class WindowedWordCount {
+ public static void main(String[] args) throws Exception {
+ StreamBuilder builder = new StreamBuilder();
+ // A stream of random sentences
+ builder.newStream(new RandomSentenceSpout(), new ValueMapper(0))
+ /*
+ * Increase the parallelism of this stream. Further operations
+ * on this stream will execute at this level of parallelism.
+ */
+ .repartition(2)
+ /*
+ * split the sentences to words
+ */
+ .flatMap(s -> Arrays.asList(s.split(" ")))
+ /*
+ * create a stream of (word, 1) pairs
+ */
+ .mapToPair(w -> Pair.of(w, 1))
+ /*
+ * group by word so that the same words end up in the same partition
+ */
+ .groupByKey()
+ /*
+ * a two seconds tumbling window
+ */
+ .window(TumblingWindows.of(Duration.seconds(2)))
+ /*
+ * compute the word counts in the last two second window
+ */
+ .aggregateByKey(new Count<>())
+ /*
+ * emit the count for the words that occurred
+ * at-least five times in the last two seconds
+ */
+ .filter(x -> x.getSecond() >= 5)
+ /*
+ * print the results to stdout
+ */
+ .print();
+
+ Config config = new Config();
+ if (args.length > 0) {
+ config.setNumWorkers(1);
+ StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build());
+ } else {
+ LocalCluster cluster = new LocalCluster();
+ cluster.submitTopology("test", config, builder.build());
+ Utils.sleep(60000);
+ cluster.killTopology("test");
+ cluster.shutdown();
+ }
+ }
+}
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java
new file mode 100644
index 00000000000..a7116963d37
--- /dev/null
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java
@@ -0,0 +1,111 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.starter.streams;
+
+import org.apache.storm.Config;
+import org.apache.storm.LocalCluster;
+import org.apache.storm.StormSubmitter;
+import org.apache.storm.redis.bolt.RedisStoreBolt;
+import org.apache.storm.redis.common.config.JedisPoolConfig;
+import org.apache.storm.redis.common.mapper.RedisDataTypeDescription;
+import org.apache.storm.redis.common.mapper.RedisStoreMapper;
+import org.apache.storm.streams.Pair;
+import org.apache.storm.streams.StreamBuilder;
+import org.apache.storm.streams.operations.aggregators.Count;
+import org.apache.storm.streams.operations.mappers.ValueMapper;
+import org.apache.storm.testing.TestWordSpout;
+import org.apache.storm.topology.IRichBolt;
+import org.apache.storm.tuple.ITuple;
+import org.apache.storm.utils.Utils;
+
+/**
+ * An example that computes word counts and finally emits the results to an
+ * external bolt (sink)
+ */
+public class WordCountToBolt {
+ public static void main(String[] args) throws Exception {
+ StreamBuilder builder = new StreamBuilder();
+
+ // Redis config parameters for the RedisStoreBolt
+ JedisPoolConfig poolConfig = new JedisPoolConfig.Builder()
+ .setHost("127.0.0.1").setPort(6379).build();
+ // Storm tuple to redis key-value mapper
+ RedisStoreMapper storeMapper = new WordCountStoreMapper();
+ // The redis bolt (sink)
+ IRichBolt redisStoreBolt = new RedisStoreBolt(poolConfig, storeMapper);
+
+ // A stream of words
+ builder.newStream(new TestWordSpout(), new ValueMapper(0))
+ /*
+ * create a stream of (word, 1) pairs
+ */
+ .mapToPair(w -> Pair.of(w, 1))
+ /*
+ * group by key (word)
+ */
+ .groupByKey()
+ /*
+ * aggregate the count
+ */
+ .aggregateByKey(new Count<>())
+ /*
+ * The result of aggregation is forwarded to
+ * the RedisStoreBolt. The forwarded tuple is a
+ * key-value pair of (word, count) with ("key", "value")
+ * being the field names.
+ */
+ .to(redisStoreBolt);
+
+ Config config = new Config();
+ if (args.length > 0) {
+ config.setNumWorkers(1);
+ StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build());
+ } else {
+ LocalCluster cluster = new LocalCluster();
+ cluster.submitTopology("test", config, builder.build());
+ Utils.sleep(60000);
+ cluster.killTopology("test");
+ cluster.shutdown();
+ }
+ }
+
+ // Maps a storm tuple to redis key and value
+ private static class WordCountStoreMapper implements RedisStoreMapper {
+ private final RedisDataTypeDescription description;
+ private final String hashKey = "wordCount";
+
+ WordCountStoreMapper() {
+ description = new RedisDataTypeDescription(RedisDataTypeDescription.RedisDataType.HASH, hashKey);
+ }
+
+ @Override
+ public RedisDataTypeDescription getDataTypeDescription() {
+ return description;
+ }
+
+ @Override
+ public String getKeyFromTuple(ITuple tuple) {
+ return tuple.getStringByField("key");
+ }
+
+ @Override
+ public String getValueFromTuple(ITuple tuple) {
+ return String.valueOf(tuple.getLongByField("value"));
+ }
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/Edge.java b/storm-core/src/jvm/org/apache/storm/streams/Edge.java
new file mode 100644
index 00000000000..9b13562717e
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/Edge.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams;
+
+import java.io.Serializable;
+
+/**
+ * An edge connects source and target nodes
+ */
+class Edge implements Serializable {
+ private final Node source;
+ private final Node target;
+
+ Edge(Node source, Node target) {
+ this.source = source;
+ this.target = target;
+ }
+
+ public Node getSource() {
+ return source;
+ }
+
+ public Node getTarget() {
+ return target;
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/GroupingInfo.java b/storm-core/src/jvm/org/apache/storm/streams/GroupingInfo.java
new file mode 100644
index 00000000000..81def4b0580
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/GroupingInfo.java
@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams;
+
+import org.apache.storm.topology.BoltDeclarer;
+import org.apache.storm.tuple.Fields;
+
+import java.io.Serializable;
+
+abstract class GroupingInfo implements Serializable {
+ private final Fields fields;
+
+ private GroupingInfo() {
+ this(null);
+ }
+
+ private GroupingInfo(Fields fields) {
+ this.fields = fields;
+ }
+
+ public abstract void declareGrouping(BoltDeclarer declarer, String componentId, String streamId, Fields fields);
+
+ public static GroupingInfo shuffle() {
+ return new GroupingInfo() {
+ @Override
+ public void declareGrouping(BoltDeclarer declarer, String componentId, String streamId, Fields fields) {
+ declarer.shuffleGrouping(componentId, streamId);
+ }
+ };
+ }
+
+ public static GroupingInfo fields(Fields fields) {
+ return new GroupingInfo(fields) {
+ @Override
+ public void declareGrouping(BoltDeclarer declarer, String componentId, String streamId, Fields fields) {
+ declarer.fieldsGrouping(componentId, streamId, fields);
+ }
+ };
+ }
+
+ public static GroupingInfo global() {
+ return new GroupingInfo() {
+ @Override
+ public void declareGrouping(BoltDeclarer declarer, String componentId, String streamId, Fields fields) {
+ declarer.globalGrouping(componentId, streamId);
+ }
+ };
+ }
+
+ public static GroupingInfo all() {
+ return new GroupingInfo() {
+ @Override
+ public void declareGrouping(BoltDeclarer declarer, String componentId, String streamId, Fields fields) {
+ declarer.allGrouping(componentId, streamId);
+ }
+ };
+ }
+
+ public Fields getFields() {
+ return fields;
+ }
+
+ @Override
+ public String toString() {
+ return "GroupingInfo{" +
+ "fields=" + fields +
+ '}';
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/Node.java b/storm-core/src/jvm/org/apache/storm/streams/Node.java
new file mode 100644
index 00000000000..f9de390a9c9
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/Node.java
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Multimaps;
+import org.apache.storm.generated.StreamInfo;
+import org.apache.storm.topology.IComponent;
+import org.apache.storm.topology.OutputFieldsGetter;
+import org.apache.storm.tuple.Fields;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Base class for a Node which form the vertices of the topology DAG.
+ */
+abstract class Node implements Serializable {
+ private final Set outputStreams;
+ protected final Fields outputFields;
+ protected String componentId;
+ protected int parallelism;
+ // the parent streams that this node subscribes to
+ private final Multimap parentStreams = ArrayListMultimap.create();
+
+ Node(Set outputStreams, Fields outputFields, String componentId, int parallelism) {
+ this.outputStreams = new HashSet<>(outputStreams);
+ this.outputFields = outputFields;
+ this.componentId = componentId;
+ this.parallelism = parallelism;
+ }
+
+ Node(String outputStream, Fields outputFields, String componentId, int parallelism) {
+ this(Collections.singleton(outputStream), outputFields, componentId, parallelism);
+ }
+
+ Node(String outputStream, Fields outputFields, String componentId) {
+ this(outputStream, outputFields, componentId, 1);
+ }
+
+ Node(String outputStream, Fields outputFields) {
+ this(outputStream, outputFields, null);
+ }
+
+ public Fields getOutputFields() {
+ return outputFields;
+ }
+
+ String getComponentId() {
+ return componentId;
+ }
+
+ void setComponentId(String componentId) {
+ this.componentId = componentId;
+ }
+
+ Integer getParallelism() {
+ return parallelism;
+ }
+
+ void setParallelism(int parallelism) {
+ this.parallelism = parallelism;
+ }
+
+ void addParentStream(Node parent, String streamId) {
+ parentStreams.put(parent, streamId);
+ }
+
+ void removeParentStreams(Node parent) {
+ parentStreams.removeAll(parent);
+ }
+
+ Set getOutputStreams() {
+ return Collections.unmodifiableSet(outputStreams);
+ }
+
+ Collection getParentStreams(Node parent) {
+ return parentStreams.get(parent);
+ }
+
+ Set getParents(String stream) {
+ Multimap rev = Multimaps.invertFrom(parentStreams, ArrayListMultimap.create());
+ return new HashSet<>(rev.get(stream));
+ }
+
+ void addOutputStream(String streamId) {
+ outputStreams.add(streamId);
+ }
+
+ static Fields getOutputFields(IComponent component, String streamId) {
+ OutputFieldsGetter getter = new OutputFieldsGetter();
+ component.declareOutputFields(getter);
+ Map fieldsDeclaration = getter.getFieldsDeclaration();
+ if ((fieldsDeclaration != null) && fieldsDeclaration.containsKey(streamId)) {
+ return new Fields(fieldsDeclaration.get(streamId).get_output_fields());
+ }
+ return new Fields();
+ }
+
+ @Override
+ public String toString() {
+ return "Node{" +
+ "outputStreams='" + outputStreams + '\'' +
+ ", outputFields=" + outputFields +
+ ", componentId='" + componentId + '\'' +
+ ", parallelism=" + parallelism +
+ '}';
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/Pair.java b/storm-core/src/jvm/org/apache/storm/streams/Pair.java
new file mode 100644
index 00000000000..00443598510
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/Pair.java
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams;
+
+import java.io.Serializable;
+
+/**
+ * A pair of values.
+ *
+ * @param the type of the first value
+ * @param the type of the second value
+ */
+public final class Pair implements Serializable {
+ private final T1 first;
+ private final T2 second;
+
+ /**
+ * Constructs a new pair of values
+ *
+ * @param first the first value
+ * @param second the second value
+ */
+ private Pair(T1 first, T2 second) {
+ this.first = first;
+ this.second = second;
+ }
+
+ /**
+ * Returns the first value in a pair.
+ *
+ * @return the first value
+ */
+ public T1 getFirst() {
+ return first;
+ }
+
+ /**
+ * Returns the second value in a pair.
+ *
+ * @return the second value
+ */
+ public T2 getSecond() {
+ return second;
+ }
+
+ /**
+ * Constructs a new pair of values.
+ *
+ * @param first the first value
+ * @param second the second value
+ * @param the type of the first value
+ * @param the type of the second value
+ * @return a new pair of values
+ */
+ public static Pair of(T1 first, T2 second) {
+ return new Pair<>(first, second);
+ }
+
+ @Override
+ public String toString() {
+ return "(" + first + ", " + second + ')';
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/PairStream.java b/storm-core/src/jvm/org/apache/storm/streams/PairStream.java
new file mode 100644
index 00000000000..2d18b30536f
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/PairStream.java
@@ -0,0 +1,241 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams;
+
+import org.apache.storm.Config;
+import org.apache.storm.streams.operations.Aggregator;
+import org.apache.storm.streams.operations.Consumer;
+import org.apache.storm.streams.operations.FlatMapFunction;
+import org.apache.storm.streams.operations.Function;
+import org.apache.storm.streams.operations.PairValueJoiner;
+import org.apache.storm.streams.operations.Predicate;
+import org.apache.storm.streams.operations.Reducer;
+import org.apache.storm.streams.operations.ValueJoiner;
+import org.apache.storm.streams.processors.AggregateByKeyProcessor;
+import org.apache.storm.streams.processors.FlatMapValuesProcessor;
+import org.apache.storm.streams.processors.JoinProcessor;
+import org.apache.storm.streams.processors.MapValuesProcessor;
+import org.apache.storm.streams.processors.ReduceByKeyProcessor;
+import org.apache.storm.streams.processors.UpdateStateByKeyProcessor;
+import org.apache.storm.streams.windowing.Window;
+import org.apache.storm.tuple.Fields;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Represents a stream of key-value pairs.
+ *
+ * @param the key type
+ * @param the value type
+ */
+public class PairStream extends Stream> {
+
+ PairStream(StreamBuilder topology, Node node) {
+ super(topology, node);
+ }
+
+ /**
+ * 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 the result type
+ * @return the new stream
+ */
+ public PairStream mapValues(Function super V, ? extends R> function) {
+ return new PairStream<>(streamBuilder, addProcessorNode(new MapValuesProcessor<>(function), KEY_VALUE));
+ }
+
+ /**
+ * Return a new stream by applying a {@link FlatMapFunction} function to the value of each key-value pairs in
+ * this stream.
+ *
+ * @param function the flatmap function
+ * @param the result type
+ * @return the new stream
+ */
+ public PairStream flatMapValues(FlatMapFunction function) {
+ return new PairStream<>(streamBuilder, addProcessorNode(new FlatMapValuesProcessor<>(function), KEY_VALUE));
+ }
+
+ /**
+ * Aggregates the values for each key of this stream using the given {@link Aggregator}.
+ *
+ * @param aggregator the aggregator
+ * @param the result type
+ * @return the new stream
+ */
+ public PairStream aggregateByKey(Aggregator super V, ? extends R> aggregator) {
+ return new PairStream<>(streamBuilder, addProcessorNode(new AggregateByKeyProcessor<>(aggregator), KEY_VALUE));
+ }
+
+ /**
+ * Performs a reduction on the values for each key of this stream by repeatedly applying the reducer.
+ *
+ * @param reducer the reducer
+ * @return the new stream
+ */
+ public PairStream reduceByKey(Reducer reducer) {
+ return new PairStream<>(streamBuilder, addProcessorNode(new ReduceByKeyProcessor<>(reducer), KEY_VALUE));
+ }
+
+ /**
+ * Returns a new stream where the values are grouped by the keys.
+ *
+ * @return the new stream
+ */
+ public PairStream groupByKey() {
+ return partitionBy(KEY);
+ }
+
+ /**
+ * 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> groupByKeyAndWindow(Window, ?> window) {
+ return groupByKey().window(window).aggregateByKey(new MergeValues<>());
+ }
+
+ /**
+ * 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 reduceByKeyAndWindow(Reducer reducer, Window, ?> window) {
+ return groupByKey().window(window).reduceByKey(reducer);
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public PairStream peek(Consumer super Pair> action) {
+ return toPairStream(super.peek(action));
+ }
+
+ /**
+ * Join the values of this stream with the values having the same key from the other stream.
+ *
+ * Note: The parallelism and windowing parameters (if windowed) of this stream is carried forward to the joined stream.
+ *
+ *
+ * @param otherStream the other stream
+ * @param the type of the values in the other stream
+ * @return the new stream
+ */
+ public PairStream> join(PairStream otherStream) {
+ return join(otherStream, new PairValueJoiner<>());
+ }
+
+ /**
+ * Join the values of this stream with the values having the same key from the other stream.
+ *
+ * Note: The parallelism and windowing parameters (if windowed) of this stream is carried forward to the joined stream.
+ *
+ *
+ * @param otherStream the other stream
+ * @param valueJoiner the {@link ValueJoiner}
+ * @param the type of the values resulting from the join
+ * @param the type of the values in the other stream
+ * @return the new stream
+ */
+ public PairStream join(PairStream otherStream,
+ ValueJoiner super V, ? super V1, ? extends R> valueJoiner) {
+ String leftStream = stream;
+ String rightStream = otherStream.stream;
+ Node joinNode = addProcessorNode(new JoinProcessor<>(leftStream, rightStream, valueJoiner), KEY_VALUE);
+ addNode(otherStream.getNode(), joinNode, joinNode.getParallelism());
+ return new PairStream<>(streamBuilder, joinNode);
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public PairStream window(Window, ?> window) {
+ return toPairStream(super.window(window));
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public PairStream repartition(int parallelism) {
+ return toPairStream(super.repartition(parallelism));
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ @SuppressWarnings("unchecked")
+ public PairStream[] branch(Predicate>... predicates) {
+ List> pairStreams = new ArrayList<>();
+ for (Stream> stream : super.branch(predicates)) {
+ pairStreams.add(toPairStream(stream));
+ }
+ return pairStreams.toArray(new PairStream[pairStreams.size()]);
+ }
+
+ /**
+ * Update the state by applying the given aggregator 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 aggregator the aggregator
+ * @param the result type
+ * @return the {@link StreamState} which can be used to query the state
+ */
+ public StreamState updateStateByKey(Aggregator super V, ? extends R> aggregator) {
+ return new StreamState<>(
+ new PairStream<>(streamBuilder, addProcessorNode(new UpdateStateByKeyProcessor<>(aggregator), KEY_VALUE)));
+ }
+
+ private PairStream partitionBy(Fields fields) {
+ return new PairStream<>(
+ streamBuilder,
+ addNode(new PartitionNode(stream, node.getOutputFields(), GroupingInfo.fields(fields))));
+ }
+
+ private PairStream toPairStream(Stream> stream) {
+ return new PairStream<>(stream.streamBuilder, stream.node);
+ }
+
+ // used internally to merge values in groupByKeyAndWindow
+ private static class MergeValues implements Aggregator> {
+ @Override
+ public ArrayList init() {
+ return new ArrayList<>();
+ }
+
+ @Override
+ public ArrayList apply(V value, ArrayList aggregate) {
+ aggregate.add(value);
+ return aggregate;
+ }
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/PartitionNode.java b/storm-core/src/jvm/org/apache/storm/streams/PartitionNode.java
new file mode 100644
index 00000000000..ca92deff2a9
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/PartitionNode.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams;
+
+import org.apache.storm.tuple.Fields;
+
+/**
+ * Node that holds partitioning/grouping information.
+ * This is used for operations like groupBy (fields grouping), global
+ * aggregate/reduce (global grouping), state query (all grouping).
+ */
+class PartitionNode extends Node {
+ private final GroupingInfo groupingInfo;
+
+ PartitionNode(String outputStream, Fields outputFields, GroupingInfo groupingInfo) {
+ super(outputStream, outputFields);
+ this.groupingInfo = groupingInfo;
+ }
+
+ PartitionNode(String outputStream, Fields outputFields) {
+ this(outputStream, outputFields, GroupingInfo.shuffle());
+ }
+
+ GroupingInfo getGroupingInfo() {
+ return groupingInfo;
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/ProcessorBolt.java b/storm-core/src/jvm/org/apache/storm/streams/ProcessorBolt.java
new file mode 100644
index 00000000000..f1163ca7d08
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/ProcessorBolt.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams;
+
+import com.google.common.collect.Multimap;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseRichBolt;
+import org.apache.storm.tuple.Tuple;
+import org.jgrapht.DirectedGraph;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Stream bolt that executes the different processors (except windowed and stateful operations)
+ */
+class ProcessorBolt extends BaseRichBolt implements StreamBolt {
+ private final ProcessorBoltDelegate delegate;
+
+ ProcessorBolt(String id, DirectedGraph graph, List nodes) {
+ delegate = new ProcessorBoltDelegate(id, graph, nodes);
+ }
+
+ @Override
+ public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+ delegate.prepare(stormConf, context, collector);
+ }
+
+ @Override
+ public void execute(Tuple input) {
+ delegate.processAndAck(input);
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ delegate.declareOutputFields(declarer);
+ }
+
+
+ @Override
+ public void setTimestampField(String fieldName) {
+ delegate.setTimestampField(fieldName);
+ }
+
+ @Override
+ public String getId() {
+ return delegate.getId();
+ }
+
+ void setStreamToInitialProcessors(Multimap streamToInitialProcessors) {
+ delegate.setStreamToInitialProcessors(streamToInitialProcessors);
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java b/storm-core/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java
new file mode 100644
index 00000000000..5bc6fff1a5f
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java
@@ -0,0 +1,285 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.apache.storm.streams.processors.ChainedProcessorContext;
+import org.apache.storm.streams.processors.EmittingProcessorContext;
+import org.apache.storm.streams.processors.ForwardingProcessorContext;
+import org.apache.storm.streams.processors.Processor;
+import org.apache.storm.streams.processors.ProcessorContext;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.jgrapht.DirectedGraph;
+import org.jgrapht.graph.DirectedSubgraph;
+import org.jgrapht.traverse.TopologicalOrderIterator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+class ProcessorBoltDelegate implements Serializable {
+ private static final Logger LOG = LoggerFactory.getLogger(ProcessorBoltDelegate.class);
+ private final String id;
+ private final DirectedGraph graph;
+ private final List nodes;
+ private Map stormConf;
+ private TopologyContext topologyContext;
+ private OutputCollector outputCollector;
+ private final List outgoingProcessors = new ArrayList<>();
+ private final Set emittingProcessorContexts = new HashSet<>();
+ private final Map> punctuationState = new HashMap<>();
+ private Multimap streamToInitialProcessors;
+ private String timestampField;
+
+ ProcessorBoltDelegate(String id, DirectedGraph graph, List nodes) {
+ this.id = id;
+ this.graph = graph;
+ this.nodes = new ArrayList<>(nodes);
+ }
+
+ String getId() {
+ return id;
+ }
+
+ void addNodes(Collection nodes) {
+ this.nodes.addAll(nodes);
+ }
+
+ List getNodes() {
+ return Collections.unmodifiableList(nodes);
+ }
+
+ void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+ this.stormConf = stormConf;
+ topologyContext = context;
+ outputCollector = collector;
+ DirectedSubgraph subgraph = new DirectedSubgraph<>(graph, new HashSet<>(nodes), null);
+ TopologicalOrderIterator it = new TopologicalOrderIterator<>(subgraph);
+ while (it.hasNext()) {
+ Node node = it.next();
+ if (!(node instanceof ProcessorNode)) {
+ throw new IllegalStateException("Not a processor node " + node);
+ }
+ ProcessorNode processorNode = (ProcessorNode) node;
+ List children = StreamUtil.getChildren(subgraph, processorNode);
+ ProcessorContext processorContext;
+ if (children.isEmpty()) {
+ processorContext = createEmittingContext(processorNode);
+ } else {
+ Multimap streamToChildren = ArrayListMultimap.create();
+ for (ProcessorNode child : children) {
+ for (String stream : child.getParentStreams(processorNode)) {
+ streamToChildren.put(stream, child);
+ }
+ }
+ ForwardingProcessorContext forwardingContext = new ForwardingProcessorContext(processorNode, streamToChildren);
+ if (hasOutgoingChild(processorNode, new HashSet<>(children))) {
+ processorContext = new ChainedProcessorContext(processorNode, forwardingContext, createEmittingContext(processorNode));
+ } else {
+ processorContext = forwardingContext;
+ }
+ }
+ processorNode.initProcessorContext(processorContext);
+ }
+ if (timestampField != null) {
+ for (EmittingProcessorContext ctx : emittingProcessorContexts) {
+ ctx.setTimestampField(timestampField);
+ }
+ }
+ }
+
+ void declareOutputFields(OutputFieldsDeclarer declarer) {
+ for (ProcessorNode node : nodes) {
+ for (String stream : node.getOutputStreams()) {
+ if (timestampField == null) {
+ declarer.declareStream(stream, node.getOutputFields());
+ } else {
+ List fields = new ArrayList<>();
+ fields.addAll(node.getOutputFields().toList());
+ fields.add(timestampField);
+ declarer.declareStream(stream, new Fields(fields));
+ }
+ }
+ }
+ }
+
+ void setAnchor(RefCountedTuple tuple) {
+ for (EmittingProcessorContext ctx : emittingProcessorContexts) {
+ ctx.setAnchor(tuple);
+ }
+ }
+
+ Pair
+ *
+ * @see org.apache.storm.streams.windowing.SlidingWindows
+ * @see org.apache.storm.streams.windowing.TumblingWindows
+ * @param window the window configuration
+ * @return the new stream
+ */
+ public Stream window(Window, ?> window) {
+ return new Stream<>(streamBuilder, addNode(new WindowNode(window, stream, node.getOutputFields())));
+ }
+
+ /**
+ * Performs an action for each element of this stream.
+ *
+ * @param action an action to perform on the elements
+ */
+ public void forEach(Consumer super T> action) {
+ addProcessorNode(new ForEachProcessor<>(action), new Fields());
+ }
+
+ /**
+ * Returns a stream consisting of the elements of this stream, additionally performing the provided action on
+ * each element as they are consumed from the resulting stream.
+ *
+ * @param action the action to perform on the element as they are consumed from the stream
+ * @return the new stream
+ */
+ public Stream peek(Consumer super T> action) {
+ return new Stream<>(streamBuilder, addProcessorNode(new PeekProcessor<>(action), node.getOutputFields()));
+ }
+
+ /**
+ * Aggregates the values in this stream using the aggregator. This does a global aggregation, i.e. the elements
+ * across all the partitions are forwarded to a single task for computing the aggregate.
+ *
+ * If the stream is windowed, the aggregate result is emitted after each window activation and represents the
+ * aggregate of elements that fall within that window.
+ * If the stream is not windowed, the aggregate result is emitted as each new element in the stream is processed.
+ *
+ *
+ * @param aggregator the aggregator
+ * @param the result type
+ * @return the new stream
+ */
+ public Stream aggregate(Aggregator super T, ? extends R> aggregator) {
+ return new Stream<>(streamBuilder, global().addProcessorNode(new AggregateProcessor<>(aggregator), VALUE));
+ }
+
+ /**
+ * Performs a reduction on the elements of this stream, by repeatedly applying the reducer.
+ *
+ * If the stream is windowed, the result is emitted after each window activation and represents the
+ * reduction of elements that fall within that window.
+ * If the stream is not windowed, the result is emitted as each new element in the stream is processed.
+ *
+ *
+ * @param reducer the reducer
+ * @return the new stream
+ */
+ public Stream reduce(Reducer reducer) {
+ return new Stream<>(streamBuilder, global().addProcessorNode(new ReduceProcessor<>(reducer), VALUE));
+ }
+
+ /**
+ * Returns a new stream with the given value of parallelism. Further operations on this stream
+ * would execute at this level of parallelism.
+ *
+ * @param parallelism the parallelism value
+ * @return the new stream
+ */
+ public Stream repartition(int parallelism) {
+ if (parallelism < 1) {
+ throw new IllegalArgumentException("Parallelism should be >= 1");
+ }
+ Node partitionNode = addNode(node, new PartitionNode(stream, node.getOutputFields()), parallelism);
+ return new Stream<>(streamBuilder, partitionNode);
+ }
+
+ /**
+ * Returns an array of streams by splitting the given stream into multiple branches based on the given
+ * predicates. The predicates are applied in the given order to the values of this stream and the result
+ * is forwarded to the corresponding (index based) result stream based on the (index of) predicate that matches.
+ * If none of the predicates match a value, that value is dropped.
+ *
+ * @param predicates the predicates
+ * @return an array of result streams (branches) corresponding to the given predicates
+ */
+ @SuppressWarnings("unchecked")
+ public Stream[] branch(Predicate... predicates) {
+ List> childStreams = new ArrayList<>();
+ if (predicates.length > 0) {
+ BranchProcessor branchProcessor = new BranchProcessor<>();
+ Node branchNode = addProcessorNode(branchProcessor, VALUE);
+ for (Predicate predicate : predicates) {
+ // create a child node (identity) per branch
+ ProcessorNode child = makeProcessorNode(new MapProcessor<>(new IdentityFunction<>()), node.getOutputFields());
+ String branchStream = child.getOutputStreams().iterator().next() + "-branch";
+ // branchStream is the parent stream that connects branch processor to this child
+ branchNode.addOutputStream(branchStream);
+ addNode(branchNode, child, branchStream);
+ childStreams.add(new Stream<>(streamBuilder, child));
+ branchProcessor.addPredicate(predicate, branchStream);
+ }
+ }
+ return childStreams.toArray((Stream[]) new Stream[childStreams.size()]);
+ }
+
+ /**
+ * Print the values in this stream.
+ */
+ public void print() {
+ forEach(new PrintConsumer());
+ }
+
+ /**
+ * Sends the elements of this stream to a bolt. This could be used to plug in existing bolts as
+ * sinks in the stream, for e.g. a {@code RedisStoreBolt}. The bolt would have a parallelism of 1.
+ *
+ * Note: This would provide guarantees only based on what the bolt provides.
+ *
+ *
+ * @param bolt the bolt
+ */
+ public void to(IRichBolt bolt) {
+ to(bolt, 1);
+ }
+
+ /**
+ * Sends the elements of this stream to a bolt. This could be used to plug in existing bolts as
+ * sinks in the stream, for e.g. a {@code RedisStoreBolt}.
+ *
+ * Note: This would provide guarantees only based on what the bolt provides.
+ *
+ *
+ * @param bolt the bolt
+ * @param parallelism the parallelism of the bolt
+ */
+ public void to(IRichBolt bolt, int parallelism) {
+ addSinkNode(new SinkNode(bolt), parallelism);
+ }
+
+ /**
+ * Sends the elements of this stream to a bolt. This could be used to plug in existing bolts as
+ * sinks in the stream, for e.g. a {@code RedisStoreBolt}. The bolt would have a parallelism of 1.
+ *
+ * Note: This would provide guarantees only based on what the bolt provides.
+ *
+ *
+ * @param bolt the bolt
+ */
+ public void to(IBasicBolt bolt) {
+ to(bolt, 1);
+ }
+
+ /**
+ * Sends the elements of this stream to a bolt. This could be used to plug in existing bolts as
+ * sinks in the stream, for e.g. a {@code RedisStoreBolt}.
+ *
+ * Note: This would provide guarantees only based on what the bolt provides.
+ *
+ *
+ * @param bolt the bolt
+ * @param parallelism the parallelism of the bolt
+ */
+ public void to(IBasicBolt bolt, int parallelism) {
+ addSinkNode(new SinkNode(bolt), parallelism);
+ }
+
+ /**
+ * Queries the given stream state with the values in this stream as the keys.
+ *
+ * @param streamState the stream state
+ * @param the value type
+ * @return the result stream
+ */
+ public PairStream stateQuery(StreamState streamState) {
+ // need all grouping for state query since the state is local
+ Node node = all().addProcessorNode(new StateQueryProcessor<>(streamState), KEY_VALUE);
+ // add 'updateState' node as parent so that state query gets processed after update state
+ addNode(streamState.getUpdateStateNode(), node, node.getParallelism());
+ return new PairStream<>(streamBuilder, node);
+ }
+
+ Node getNode() {
+ return node;
+ }
+
+ Node addNode(Node parent, Node child, int parallelism) {
+ return streamBuilder.addNode(parent, child, parallelism);
+ }
+
+ Node addNode(Node child) {
+ return addNode(this.node, child);
+ }
+
+ Node addProcessorNode(Processor> processor, Fields outputFields) {
+ return addNode(makeProcessorNode(processor, outputFields));
+ }
+
+ String getStream() {
+ return stream;
+ }
+
+ private Node addNode(Node parent, Node child) {
+ return streamBuilder.addNode(parent, child);
+ }
+
+ private Node addNode(Node parent, Node child, String parentStreamId) {
+ return streamBuilder.addNode(parent, child, parentStreamId);
+ }
+
+ private Node addNode(Node child, int parallelism, String parentStreamId) {
+ return streamBuilder.addNode(this.node, child, parallelism, parentStreamId);
+ }
+
+ private ProcessorNode makeProcessorNode(Processor> processor, Fields outputFields) {
+ return new ProcessorNode(processor, UniqueIdGen.getInstance().getUniqueStreamId(), outputFields);
+ }
+
+ private void addSinkNode(SinkNode sinkNode, int parallelism) {
+ String boltId = UniqueIdGen.getInstance().getUniqueBoltId();
+ sinkNode.setComponentId(boltId);
+ sinkNode.setParallelism(parallelism);
+ if (node instanceof SpoutNode) {
+ addNode(sinkNode, parallelism, Utils.DEFAULT_STREAM_ID);
+ } else {
+ /*
+ * add a stream__sink stream to the current node (parent) for emitting
+ * just the values (no punctuation) to the bolt.
+ */
+ String sinkStream = StreamUtil.getSinkStream(stream);
+ node.addOutputStream(sinkStream);
+ addNode(sinkNode, parallelism, sinkStream);
+ }
+ }
+
+ private Stream global() {
+ Node partitionNode = addNode(new PartitionNode(stream, node.getOutputFields(), GroupingInfo.global()));
+ return new Stream<>(streamBuilder, partitionNode);
+ }
+
+ private Stream all() {
+ Node partitionNode = addNode(new PartitionNode(stream, node.getOutputFields(), GroupingInfo.all()));
+ return new Stream<>(streamBuilder, partitionNode);
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/StreamBolt.java b/storm-core/src/jvm/org/apache/storm/streams/StreamBolt.java
new file mode 100644
index 00000000000..6170ea8703d
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/StreamBolt.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams;
+
+/**
+ * Interface for bolts that executes the streaming operations via the
+ * processors.
+ */
+interface StreamBolt {
+ /**
+ * The bolt component id. This is auto generated value like (bolt-N).
+ *
+ * @return the bolt id.
+ */
+ String getId();
+
+ /**
+ * Set the timestamp field name for event time based processing.
+ *
+ * @param fieldName the timestamp field name
+ */
+ void setTimestampField(String fieldName);
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java b/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java
new file mode 100644
index 00000000000..e19a0c63bda
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java
@@ -0,0 +1,591 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams;
+
+import com.google.common.base.Function;
+import com.google.common.base.Predicate;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Collections2;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Table;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.streams.operations.IdentityFunction;
+import org.apache.storm.streams.operations.mappers.PairValueMapper;
+import org.apache.storm.streams.operations.mappers.TupleValueMapper;
+import org.apache.storm.streams.processors.JoinProcessor;
+import org.apache.storm.streams.processors.MapProcessor;
+import org.apache.storm.streams.processors.StateQueryProcessor;
+import org.apache.storm.streams.processors.StatefulProcessor;
+import org.apache.storm.streams.windowing.Window;
+import org.apache.storm.topology.BoltDeclarer;
+import org.apache.storm.topology.IBasicBolt;
+import org.apache.storm.topology.IComponent;
+import org.apache.storm.topology.IRichBolt;
+import org.apache.storm.topology.IRichSpout;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.tuple.Tuple;
+import org.jgrapht.graph.DefaultDirectedGraph;
+import org.jgrapht.traverse.TopologicalOrderIterator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Set;
+
+/**
+ * A builder for constructing a {@link StormTopology} via storm streams api (DSL)
+ */
+public class StreamBuilder {
+ private static final Logger LOG = LoggerFactory.getLogger(StreamBuilder.class);
+ private final DefaultDirectedGraph graph;
+ private final Table nodeGroupingInfo = HashBasedTable.create();
+ private final Map windowInfo = new HashMap<>();
+ private final List curGroup = new ArrayList<>();
+ private int statefulProcessorCount = 0;
+ private final Map streamBolts = new HashMap<>();
+ private String timestampFieldName = null;
+
+ /**
+ * Creates a new {@link StreamBuilder}
+ */
+ public StreamBuilder() {
+ graph = new DefaultDirectedGraph<>(new StreamsEdgeFactory());
+ }
+
+ /**
+ * Creates a new {@link Stream} of tuples from the given {@link IRichSpout}
+ *
+ * @param spout the spout
+ * @return the new stream
+ */
+ public Stream newStream(IRichSpout spout) {
+ return newStream(spout, 1);
+ }
+
+ /**
+ * Creates a new {@link Stream} of tuples from the given {@link IRichSpout} with the given
+ * parallelism.
+ *
+ * @param spout the spout
+ * @param parallelism the parallelism of the stream
+ * @return the new stream
+ */
+ public Stream newStream(IRichSpout spout, int parallelism) {
+ SpoutNode spoutNode = new SpoutNode(spout);
+ String spoutId = UniqueIdGen.getInstance().getUniqueSpoutId();
+ spoutNode.setComponentId(spoutId);
+ spoutNode.setParallelism(parallelism);
+ graph.addVertex(spoutNode);
+ return new Stream<>(this, spoutNode);
+ }
+
+ /**
+ * Creates a new {@link Stream} of values from the given {@link IRichSpout} by extracting field(s)
+ * from tuples via the supplied {@link TupleValueMapper}.
+ *
+ * @param spout the spout
+ * @param valueMapper the value mapper
+ * @param the type of values in the resultant stream
+ * @return the new stream
+ */
+ public Stream newStream(IRichSpout spout, TupleValueMapper valueMapper) {
+ return newStream(spout).map(valueMapper);
+ }
+
+
+ /**
+ * Creates a new {@link Stream} of values from the given {@link IRichSpout} by extracting field(s)
+ * from tuples via the supplied {@link TupleValueMapper} with the given parallelism.
+ *
+ * @param spout the spout
+ * @param valueMapper the value mapper
+ * @param parallelism the parallelism of the stream
+ * @param the type of values in the resultant stream
+ * @return the new stream
+ */
+ public Stream newStream(IRichSpout spout, TupleValueMapper valueMapper, int parallelism) {
+ return newStream(spout, parallelism).map(valueMapper);
+ }
+
+ /**
+ * Creates a new {@link PairStream} of key-value pairs from the given {@link IRichSpout} by extracting key and
+ * value from tuples via the supplied {@link PairValueMapper}.
+ *
+ * @param spout the spout
+ * @param pairValueMapper the pair value mapper
+ * @param the key type
+ * @param the value type
+ * @return the new stream of key-value pairs
+ */
+ public PairStream newStream(IRichSpout spout, PairValueMapper pairValueMapper) {
+ return newStream(spout).mapToPair(pairValueMapper);
+ }
+
+ /**
+ * Creates a new {@link PairStream} of key-value pairs from the given {@link IRichSpout} by extracting key and
+ * value from tuples via the supplied {@link PairValueMapper} and with the given value of parallelism.
+ *
+ * @param spout the spout
+ * @param pairValueMapper the pair value mapper
+ * @param parallelism the parallelism of the stream
+ * @param the key type
+ * @param the value type
+ * @return the new stream of key-value pairs
+ */
+ public PairStream newStream(IRichSpout spout, PairValueMapper pairValueMapper, int parallelism) {
+ return newStream(spout, parallelism).mapToPair(pairValueMapper);
+ }
+
+
+ /**
+ * Builds a new {@link StormTopology} for the computation expressed
+ * via the stream api.
+ *
+ * @return the storm topology
+ */
+ public StormTopology build() {
+ nodeGroupingInfo.clear();
+ windowInfo.clear();
+ curGroup.clear();
+ TopologicalOrderIterator iterator = new TopologicalOrderIterator<>(graph, queue());
+ TopologyBuilder topologyBuilder = new TopologyBuilder();
+ while (iterator.hasNext()) {
+ Node node = iterator.next();
+ if (node instanceof SpoutNode) {
+ addSpout(topologyBuilder, (SpoutNode) node);
+ } else if (node instanceof ProcessorNode) {
+ handleProcessorNode((ProcessorNode) node, topologyBuilder);
+ } else if (node instanceof PartitionNode) {
+ updateNodeGroupingInfo((PartitionNode) node);
+ processCurGroup(topologyBuilder);
+ } else if (node instanceof WindowNode) {
+ updateWindowInfo((WindowNode) node);
+ processCurGroup(topologyBuilder);
+ } else if (node instanceof SinkNode) {
+ processCurGroup(topologyBuilder);
+ addSink(topologyBuilder, (SinkNode) node);
+ }
+ }
+ processCurGroup(topologyBuilder);
+ mayBeAddTsField();
+ return topologyBuilder.createTopology();
+ }
+
+ Node addNode(Node parent, Node child) {
+ return addNode(parent, child, parent.getParallelism(), parent.getOutputStreams().iterator().next());
+ }
+
+ Node addNode(Node parent, Node child, int parallelism) {
+ return addNode(parent, child, parallelism, parent.getOutputStreams().iterator().next());
+ }
+
+ Node addNode(Node parent, Node child, String parentStreamId) {
+ return addNode(parent, child, parent.getParallelism(), parentStreamId);
+ }
+
+ Node addNode(Node parent, Node child, int parallelism, String parentStreamId) {
+ graph.addVertex(child);
+ graph.addEdge(parent, child);
+ child.setParallelism(parallelism);
+ if (parent instanceof WindowNode || parent instanceof PartitionNode) {
+ child.addParentStream(parentNode(parent), parentStreamId);
+ } else {
+ child.addParentStream(parent, parentStreamId);
+ }
+ return child;
+ }
+
+ private PriorityQueue queue() {
+ // min-heap
+ return new PriorityQueue<>(new Comparator() {
+ @Override
+ public int compare(Node n1, Node n2) {
+ return getPriority(n1.getClass()) - getPriority(n2.getClass());
+ }
+
+ private int getPriority(Class extends Node> clazz) {
+ /*
+ * Nodes in the descending order of priority.
+ * ProcessorNode has the highest priority so that the topological order iterator
+ * will group as many processor nodes together as possible.
+ */
+ Class>[] p = new Class>[]{
+ ProcessorNode.class,
+ SpoutNode.class,
+ SinkNode.class,
+ PartitionNode.class,
+ WindowNode.class};
+ for (int i = 0; i < p.length; i++) {
+ if (clazz.equals(p[i])) {
+ return i;
+ }
+ }
+ return Integer.MAX_VALUE;
+ }
+ });
+ }
+
+ private void handleProcessorNode(ProcessorNode processorNode, TopologyBuilder topologyBuilder) {
+ if (processorNode.getProcessor() instanceof StatefulProcessor) {
+ statefulProcessorCount++;
+ Set initialNodes = initialProcessors(
+ curGroup.isEmpty() ? Collections.singletonList(processorNode) : curGroup);
+ Set> windows = getWindowParams(initialNodes);
+ // if we get more than one stateful operation, we need to process the
+ // current group so that we have one stateful operation per stateful bolt
+ if (statefulProcessorCount > 1 || !windows.isEmpty()) {
+ if (!curGroup.isEmpty()) {
+ processCurGroup(topologyBuilder);
+ } else if (!windows.isEmpty()) {
+ // a stateful processor immediately follows a window specification
+ splitStatefulProcessor(processorNode, topologyBuilder);
+ }
+ statefulProcessorCount = 1;
+ }
+ }
+ curGroup.add(processorNode);
+ }
+
+ /*
+ * force create a windowed bolt with identity nodes so that we don't
+ * have a stateful processor inside a windowed bolt.
+ */
+ private void splitStatefulProcessor(ProcessorNode processorNode, TopologyBuilder topologyBuilder) {
+ for (Node parent : StreamUtil.getParents(graph, processorNode)) {
+ ProcessorNode identity =
+ new ProcessorNode(new MapProcessor<>(new IdentityFunction<>()),
+ UniqueIdGen.getInstance().getUniqueStreamId(),
+ parent.getOutputFields());
+ addNode(parent, identity);
+ graph.removeEdge(parent, processorNode);
+ processorNode.removeParentStreams(parent);
+ addNode(identity, processorNode);
+ curGroup.add(identity);
+ }
+ processCurGroup(topologyBuilder);
+ }
+
+ private void mayBeAddTsField() {
+ if (timestampFieldName != null) {
+ for (StreamBolt streamBolt : streamBolts.keySet()) {
+ streamBolt.setTimestampField(timestampFieldName);
+ }
+ }
+ }
+
+ private void updateNodeGroupingInfo(PartitionNode partitionNode) {
+ if (partitionNode.getGroupingInfo() != null) {
+ for (Node parent : parentNodes(partitionNode)) {
+ for (String parentStream : partitionNode.getParentStreams(parent)) {
+ nodeGroupingInfo.put(parent, parentStream, partitionNode.getGroupingInfo());
+ }
+ }
+ }
+ }
+
+ private void updateWindowInfo(WindowNode windowNode) {
+ for (Node parent : parentNodes(windowNode)) {
+ windowInfo.put(parent, windowNode);
+ }
+ String tsField = windowNode.getWindowParams().getTimestampField();
+ if (tsField != null) {
+ if (timestampFieldName != null && !tsField.equals(timestampFieldName)) {
+ throw new IllegalArgumentException("Cannot set different timestamp field names");
+ }
+ timestampFieldName = tsField;
+ }
+ }
+
+ private Node parentNode(Node curNode) {
+ Set parentNode = parentNodes(curNode);
+ if (parentNode.size() > 1) {
+ throw new IllegalArgumentException("Node " + curNode + " has more than one parent node.");
+ }
+ if (parentNode.isEmpty()) {
+ throw new IllegalArgumentException("Node " + curNode + " has no parent.");
+ }
+ return parentNode.iterator().next();
+ }
+
+ private Set parentNodes(Node curNode) {
+ Set nodes = new HashSet<>();
+ for (Node parent : StreamUtil.getParents(graph, curNode)) {
+ if (parent instanceof ProcessorNode || parent instanceof SpoutNode) {
+ nodes.add(parent);
+ } else {
+ nodes.addAll(parentNodes(parent));
+ }
+ }
+ return nodes;
+ }
+
+ private void processCurGroup(TopologyBuilder topologyBuilder) {
+ if (curGroup.isEmpty()) {
+ return;
+ }
+
+ String boltId = UniqueIdGen.getInstance().getUniqueBoltId();
+ for (ProcessorNode processorNode : curGroup) {
+ processorNode.setComponentId(boltId);
+ processorNode.setWindowed(isWindowed(processorNode));
+ processorNode.setWindowedParentStreams(getWindowedParentStreams(processorNode));
+ }
+ final Set initialProcessors = initialProcessors(curGroup);
+ Set> windowParams = getWindowParams(initialProcessors);
+ if (windowParams.isEmpty()) {
+ if (hasStatefulProcessor(curGroup)) {
+ addStatefulBolt(topologyBuilder, boltId, initialProcessors);
+ } else {
+ addBolt(topologyBuilder, boltId, initialProcessors);
+ }
+ } else if (windowParams.size() == 1) {
+ addWindowedBolt(topologyBuilder, boltId, initialProcessors, windowParams.iterator().next());
+ } else {
+ throw new IllegalStateException("More than one window config for current group " + curGroup);
+ }
+ curGroup.clear();
+ }
+
+ private boolean hasStatefulProcessor(List processorNodes) {
+ for (ProcessorNode node : processorNodes) {
+ if (node.getProcessor() instanceof StatefulProcessor) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private int getParallelism() {
+ Set parallelisms = new HashSet<>(Collections2.transform(curGroup, new Function() {
+ @Override
+ public Integer apply(ProcessorNode input) {
+ return input.getParallelism();
+ }
+ }));
+
+ if (parallelisms.size() > 1) {
+ throw new IllegalStateException("Current group does not have same parallelism " + curGroup);
+ }
+
+ return parallelisms.isEmpty() ? 1 : parallelisms.iterator().next();
+ }
+
+ private Set> getWindowParams(Set initialProcessors) {
+ Set windowNodes = new HashSet<>();
+ Set parents;
+ for (ProcessorNode processorNode : initialProcessors) {
+ if (processorNode.getProcessor() instanceof JoinProcessor) {
+ String leftStream = ((JoinProcessor) processorNode.getProcessor()).getLeftStream();
+ parents = processorNode.getParents(leftStream);
+ } else {
+ parents = parentNodes(processorNode);
+ }
+ for (Node node : parents) {
+ if (windowInfo.containsKey(node)) {
+ windowNodes.add(windowInfo.get(node));
+ }
+ }
+ }
+
+ Set> windowParams = new HashSet<>();
+ if (!windowNodes.isEmpty()) {
+ windowParams.addAll(new HashSet<>(Collections2.transform(windowNodes, new Function>() {
+ @Override
+ public Window, ?> apply(WindowNode input) {
+ return input.getWindowParams();
+ }
+ })));
+ }
+ return windowParams;
+ }
+
+ private void addSpout(TopologyBuilder topologyBuilder, SpoutNode spout) {
+ topologyBuilder.setSpout(spout.getComponentId(), spout.getSpout(), spout.getParallelism());
+ }
+
+ private void addSink(TopologyBuilder topologyBuilder, SinkNode sinkNode) {
+ IComponent bolt = sinkNode.getBolt();
+ BoltDeclarer boltDeclarer;
+ if (bolt instanceof IRichBolt) {
+ boltDeclarer = topologyBuilder.setBolt(sinkNode.getComponentId(), (IRichBolt) bolt, sinkNode.getParallelism());
+ } else if (bolt instanceof IBasicBolt) {
+ boltDeclarer = topologyBuilder.setBolt(sinkNode.getComponentId(), (IBasicBolt) bolt, sinkNode.getParallelism());
+ } else {
+ throw new IllegalArgumentException("Expect IRichBolt or IBasicBolt in addBolt");
+ }
+ for (Node parent : parentNodes(sinkNode)) {
+ for (String stream : sinkNode.getParentStreams(parent)) {
+ declareStream(boltDeclarer, parent, stream, nodeGroupingInfo.get(parent, stream));
+ }
+ }
+ }
+
+ private StreamBolt addBolt(TopologyBuilder topologyBuilder,
+ String boltId,
+ Set initialProcessors) {
+ ProcessorBolt bolt = new ProcessorBolt(boltId, graph, curGroup);
+ BoltDeclarer boltDeclarer = topologyBuilder.setBolt(boltId, bolt, getParallelism());
+ bolt.setStreamToInitialProcessors(wireBolt(curGroup, boltDeclarer, initialProcessors));
+ streamBolts.put(bolt, boltDeclarer);
+ return bolt;
+ }
+
+ private StreamBolt addStatefulBolt(TopologyBuilder topologyBuilder,
+ String boltId,
+ Set initialProcessors) {
+ StateQueryProcessor, ?> stateQueryProcessor = getStateQueryProcessor();
+ StatefulProcessorBolt, ?> bolt;
+ if (stateQueryProcessor == null) {
+ bolt = new StatefulProcessorBolt<>(boltId, graph, curGroup);
+ BoltDeclarer boltDeclarer = topologyBuilder.setBolt(boltId, bolt, getParallelism());
+ bolt.setStreamToInitialProcessors(wireBolt(curGroup, boltDeclarer, initialProcessors));
+ streamBolts.put(bolt, boltDeclarer);
+ } else {
+ // state query is added to the existing stateful bolt
+ ProcessorNode updateStateNode = stateQueryProcessor.getStreamState().getUpdateStateNode();
+ bolt = findStatefulProcessorBolt(updateStateNode);
+ for (ProcessorNode node : curGroup) {
+ node.setComponentId(bolt.getId());
+ }
+ bolt.addNodes(curGroup);
+ bolt.addStreamToInitialProcessors(wireBolt(bolt.getNodes(), streamBolts.get(bolt), initialProcessors));
+ }
+ return bolt;
+ }
+
+ private StateQueryProcessor, ?> getStateQueryProcessor() {
+ for (ProcessorNode node : curGroup) {
+ if (node.getProcessor() instanceof StateQueryProcessor) {
+ return (StateQueryProcessor, ?>) node.getProcessor();
+ }
+ }
+ return null;
+ }
+
+ private StreamBolt addWindowedBolt(TopologyBuilder topologyBuilder,
+ String boltId,
+ Set initialProcessors,
+ Window, ?> windowParam) {
+ WindowedProcessorBolt bolt = new WindowedProcessorBolt(boltId, graph, curGroup, windowParam);
+ BoltDeclarer boltDeclarer = topologyBuilder.setBolt(boltId, bolt, getParallelism());
+ bolt.setStreamToInitialProcessors(wireBolt(curGroup, boltDeclarer, initialProcessors));
+ streamBolts.put(bolt, boltDeclarer);
+ return bolt;
+ }
+
+ private StatefulProcessorBolt, ?> findStatefulProcessorBolt(ProcessorNode updateStateNode) {
+ for (StreamBolt bolt : streamBolts.keySet()) {
+ if (bolt instanceof StatefulProcessorBolt) {
+ StatefulProcessorBolt, ?> statefulProcessorBolt = (StatefulProcessorBolt) bolt;
+ if (statefulProcessorBolt.getNodes().contains(updateStateNode)) {
+ return statefulProcessorBolt;
+ }
+ }
+ }
+ throw new IllegalArgumentException("Could not find Stateful bolt for node " + updateStateNode);
+ }
+
+ private Set getWindowedParentStreams(ProcessorNode processorNode) {
+ Set res = new HashSet<>();
+ for (Node parent : parentNodes(processorNode)) {
+ if (parent instanceof ProcessorNode) {
+ ProcessorNode pn = (ProcessorNode) parent;
+ if (pn.isWindowed()) {
+ res.addAll(Collections2.filter(pn.getOutputStreams(), new Predicate() {
+ @Override
+ public boolean apply(String input) {
+ return !StreamUtil.isSinkStream(input);
+ }
+ }));
+ }
+ }
+ }
+ return res;
+ }
+
+ private Multimap wireBolt(List curGroup,
+ BoltDeclarer boltDeclarer,
+ Set initialProcessors) {
+ LOG.debug("Wiring bolt with boltDeclarer {}, curGroup {}, initialProcessors {}, nodeGroupingInfo {}",
+ boltDeclarer, curGroup, initialProcessors, nodeGroupingInfo);
+ Multimap streamToInitialProcessor = ArrayListMultimap.create();
+ Set curSet = new HashSet<>(curGroup);
+ for (ProcessorNode curNode : initialProcessors) {
+ for (Node parent : parentNodes(curNode)) {
+ if (curSet.contains(parent)) {
+ LOG.debug("Parent {} of curNode {} is in curGroup {}", parent, curNode, curGroup);
+ } else {
+ for (String stream : curNode.getParentStreams(parent)) {
+ declareStream(boltDeclarer, parent, stream, nodeGroupingInfo.get(parent, stream));
+ // put global stream id for spouts
+ if (parent.getComponentId().startsWith("spout")) {
+ stream = parent.getComponentId() + stream;
+ }
+ streamToInitialProcessor.put(stream, curNode);
+ }
+ }
+ }
+ }
+ return streamToInitialProcessor;
+ }
+
+ private void declareStream(BoltDeclarer boltDeclarer, Node parent, String streamId, GroupingInfo grouping) {
+ if (grouping == null) {
+ boltDeclarer.shuffleGrouping(parent.getComponentId(), streamId);
+ } else {
+ grouping.declareGrouping(boltDeclarer, parent.getComponentId(), streamId, grouping.getFields());
+ }
+ }
+
+ private Set initialProcessors(List curGroup) {
+ Set nodes = new HashSet<>();
+ Set curSet = new HashSet<>(curGroup);
+ for (ProcessorNode node : curGroup) {
+ for (Node parent : parentNodes(node)) {
+ if (!(parent instanceof ProcessorNode) || !curSet.contains(parent)) {
+ nodes.add(node);
+ }
+ }
+ }
+ return nodes;
+ }
+
+ private boolean isWindowed(Node curNode) {
+ for (Node parent : StreamUtil.getParents(graph, curNode)) {
+ if (parent instanceof WindowNode) {
+ return true;
+ } else if (parent instanceof ProcessorNode) {
+ ProcessorNode p = (ProcessorNode) parent;
+ if (p.isWindowed()) {
+ return true;
+ }
+ } else {
+ return (parent instanceof PartitionNode) && isWindowed(parent);
+ }
+ }
+ return false;
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/StreamState.java b/storm-core/src/jvm/org/apache/storm/streams/StreamState.java
new file mode 100644
index 00000000000..a4633f751d3
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/StreamState.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams;
+
+import java.io.Serializable;
+
+/**
+ * A wrapper for the stream state which can be used to
+ * query the state via {@link Stream#stateQuery(StreamState)}
+ *
+ * @param the key type
+ * @param the value type
+ */
+public class StreamState implements Serializable {
+ private final transient PairStream stream;
+
+ StreamState(PairStream stream) {
+ this.stream = stream;
+ }
+
+ public PairStream toPairStream() {
+ return stream;
+ }
+
+ ProcessorNode getUpdateStateNode() {
+ return (ProcessorNode) stream.node;
+ }
+}
\ No newline at end of file
diff --git a/storm-core/src/jvm/org/apache/storm/streams/StreamUtil.java b/storm-core/src/jvm/org/apache/storm/streams/StreamUtil.java
new file mode 100644
index 00000000000..0531ff60bd3
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/StreamUtil.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams;
+
+import org.jgrapht.DirectedGraph;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class StreamUtil {
+ @SuppressWarnings("unchecked")
+ public static List getParents(DirectedGraph graph, Node node) {
+ List incoming = new ArrayList<>(graph.incomingEdgesOf(node));
+ List ret = new ArrayList<>();
+ for (Edge e : incoming) {
+ ret.add((T) e.getSource());
+ }
+ return ret;
+ }
+
+ @SuppressWarnings("unchecked")
+ public static List getChildren(DirectedGraph graph, Node node) {
+ List outgoing = new ArrayList<>(graph.outgoingEdgesOf(node));
+ List ret = new ArrayList<>();
+ for (Edge e : outgoing) {
+ ret.add((T) e.getTarget());
+ }
+ return ret;
+ }
+
+
+ public static boolean isSinkStream(String streamId) {
+ return streamId.endsWith("__sink");
+ }
+
+ public static String getSinkStream(String streamId) {
+ return streamId + "__sink";
+ }
+
+ public static boolean isPunctuation(Object value) {
+ if (value instanceof Pair) {
+ value = ((Pair) value).getFirst();
+ }
+ return WindowNode.PUNCTUATION.equals(value);
+ }
+
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/StreamsEdgeFactory.java b/storm-core/src/jvm/org/apache/storm/streams/StreamsEdgeFactory.java
new file mode 100644
index 00000000000..0078690b167
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/StreamsEdgeFactory.java
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams;
+
+import org.jgrapht.EdgeFactory;
+
+import java.io.Serializable;
+
+class StreamsEdgeFactory implements EdgeFactory, Serializable {
+ @Override
+ public Edge createEdge(Node sourceVertex, Node targetVertex) {
+ return new Edge(sourceVertex, targetVertex);
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/Tuple3.java b/storm-core/src/jvm/org/apache/storm/streams/Tuple3.java
new file mode 100644
index 00000000000..77973f2746b
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/Tuple3.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams;
+
+/**
+ * A tuple of three elements along the lines of Scala's Tuple.
+ *
+ * @param the type of the first element
+ * @param the type of the second element
+ * @param the type of the third element
+ */
+public class Tuple3 {
+ public final T1 _1;
+ public final T2 _2;
+ public final T3 _3;
+
+ /**
+ * Constructs a new tuple of three elements.
+ *
+ * @param _1 the first element
+ * @param _2 the second element
+ * @param _3 the third element
+ */
+ public Tuple3(T1 _1, T2 _2, T3 _3) {
+ this._1 = _1;
+ this._2 = _2;
+ this._3 = _3;
+ }
+
+ @Override
+ public String toString() {
+ return "(" + _1 + "," + _2 + "," + _3 + ")";
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/UniqueIdGen.java b/storm-core/src/jvm/org/apache/storm/streams/UniqueIdGen.java
new file mode 100644
index 00000000000..3cbd141d122
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/UniqueIdGen.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams;
+
+class UniqueIdGen {
+ private int streamCounter = 0;
+ private int spoutCounter = 0;
+ private int boltCounter = 0;
+ private static final UniqueIdGen instance = new UniqueIdGen();
+
+ private UniqueIdGen() {
+ }
+
+ static UniqueIdGen getInstance() {
+ return instance;
+ }
+
+ String getUniqueStreamId() {
+ streamCounter++;
+ return "s" + streamCounter;
+ }
+
+ String getUniqueBoltId() {
+ boltCounter++;
+ return "bolt" + boltCounter;
+ }
+
+ String getUniqueSpoutId() {
+ spoutCounter++;
+ return "spout" + spoutCounter;
+ }
+
+ // for unit tests
+ void reset() {
+ streamCounter = 0;
+ spoutCounter = 0;
+ boltCounter = 0;
+ }
+
+}
+
diff --git a/storm-core/src/jvm/org/apache/storm/streams/WindowNode.java b/storm-core/src/jvm/org/apache/storm/streams/WindowNode.java
new file mode 100644
index 00000000000..a0e831a8492
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/WindowNode.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams;
+
+import org.apache.storm.streams.windowing.Window;
+import org.apache.storm.tuple.Fields;
+
+/**
+ * Node that captures the windowing configurations.
+ */
+public class WindowNode extends Node {
+ private final Window, ?> windowParams;
+ public static final String PUNCTUATION = "__punctuation";
+
+ WindowNode(Window, ?> windowParams, String outputStream, Fields outputFields) {
+ super(outputStream, outputFields);
+ this.windowParams = windowParams;
+ }
+
+ Window, ?> getWindowParams() {
+ return windowParams;
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/WindowedProcessorBolt.java b/storm-core/src/jvm/org/apache/storm/streams/WindowedProcessorBolt.java
new file mode 100644
index 00000000000..397134699e8
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/WindowedProcessorBolt.java
@@ -0,0 +1,137 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams;
+
+import com.google.common.collect.Multimap;
+import org.apache.storm.streams.windowing.SlidingWindows;
+import org.apache.storm.streams.windowing.TumblingWindows;
+import org.apache.storm.streams.windowing.Window;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseWindowedBolt;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.windowing.TupleWindow;
+import org.jgrapht.DirectedGraph;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.storm.streams.WindowNode.PUNCTUATION;
+
+/**
+ * Stream bolt that executes windowing operations.
+ */
+class WindowedProcessorBolt extends BaseWindowedBolt implements StreamBolt {
+ private static final Logger LOG = LoggerFactory.getLogger(WindowedProcessorBolt.class);
+ private final ProcessorBoltDelegate delegate;
+ private final Window, ?> window;
+
+ WindowedProcessorBolt(String id, DirectedGraph graph,
+ List nodes,
+ Window, ?> window) {
+ delegate = new ProcessorBoltDelegate(id, graph, nodes);
+ this.window = window;
+ setWindowConfig();
+ }
+
+ @Override
+ public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+ delegate.prepare(stormConf, context, collector);
+ }
+
+ @Override
+ public void execute(TupleWindow inputWindow) {
+ LOG.trace("Window triggered at {}, inputWindow {}", new Date(), inputWindow);
+ if (delegate.isEventTimestamp()) {
+ delegate.setEventTimestamp(inputWindow.getTimestamp());
+ }
+ for (Tuple tuple : inputWindow.get()) {
+ Pair valueAndStream = delegate.getValueAndStream(tuple);
+ if (!StreamUtil.isPunctuation(valueAndStream.getFirst())) {
+ delegate.process(valueAndStream.getFirst(), valueAndStream.getSecond());
+ }
+ }
+ for (String stream : delegate.getInitialStreams()) {
+ delegate.process(PUNCTUATION, stream);
+ }
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ delegate.declareOutputFields(declarer);
+ }
+
+ @Override
+ public void setTimestampField(String fieldName) {
+ delegate.setTimestampField(fieldName);
+ }
+
+ @Override
+ public String getId() {
+ return delegate.getId();
+ }
+
+ 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());
+ }
+ }
+
+ private void setSlidingWindowParams(Object windowLength, Object slidingInterval) {
+ if (windowLength instanceof Count) {
+ if (slidingInterval instanceof Count) {
+ withWindow((Count) windowLength, (Count) slidingInterval);
+ } else if (slidingInterval instanceof Duration) {
+ withWindow((Count) windowLength, (Duration) slidingInterval);
+ }
+ } else if (windowLength instanceof Duration) {
+ if (slidingInterval instanceof Count) {
+ withWindow((Duration) windowLength, (Count) slidingInterval);
+ } else if (slidingInterval instanceof Duration) {
+ withWindow((Duration) windowLength, (Duration) slidingInterval);
+ }
+ }
+ }
+
+ private void setTumblingWindowParams(Object windowLength) {
+ if (windowLength instanceof Count) {
+ withTumblingWindow((Count) windowLength);
+ } else if (windowLength instanceof Duration) {
+ withTumblingWindow((Duration) windowLength);
+ }
+ }
+
+ void setStreamToInitialProcessors(Multimap streamToInitialProcessors) {
+ delegate.setStreamToInitialProcessors(streamToInitialProcessors);
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/Aggregator.java b/storm-core/src/jvm/org/apache/storm/streams/operations/Aggregator.java
new file mode 100644
index 00000000000..e3feaf4e3a0
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/Aggregator.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.operations;
+
+/**
+ * Interface for aggregating values.
+ *
+ * @param the original value type
+ * @param the aggregated value type
+ */
+public interface Aggregator extends Operation {
+ /**
+ * The initial value of the aggregate to start with.
+ *
+ * @return the initial value
+ */
+ R init();
+
+ /**
+ * Returns a new aggregate by applying the value with the current aggregate.
+ *
+ * @param value the value to aggregate
+ * @param aggregate the current aggregate
+ * @return the new aggregate
+ */
+ R apply(T value, R aggregate);
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/Consumer.java b/storm-core/src/jvm/org/apache/storm/streams/operations/Consumer.java
new file mode 100644
index 00000000000..84653abb2fb
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/Consumer.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.operations;
+
+/**
+ * Represents an operation that accepts a single input argument and returns no result.
+ *
+ * @param the type of the input argument
+ */
+public interface Consumer extends Operation {
+ /**
+ * Performs an operation on the given argument.
+ *
+ * @param input the input
+ */
+ void accept(T input);
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/FlatMapFunction.java b/storm-core/src/jvm/org/apache/storm/streams/operations/FlatMapFunction.java
new file mode 100644
index 00000000000..bcacd082f85
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/FlatMapFunction.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.operations;
+
+/**
+ * A function that accepts one argument and returns an {@link Iterable} of elements as its result.
+ *
+ * @param the type of the input to the function
+ * @param the result type. An iterable of this type is returned from this function
+ */
+public interface FlatMapFunction extends Function> {
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/Function.java b/storm-core/src/jvm/org/apache/storm/streams/operations/Function.java
new file mode 100644
index 00000000000..7cef0a6b2e6
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/Function.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.operations;
+
+/**
+ * Represents a function that accepts one argument and produces a result.
+ *
+ * @param the type of the input to the function
+ * @param the type of the result of the function
+ */
+public interface Function extends Operation {
+ /**
+ * Applies this function to the given argument.
+ *
+ * @param input the input to the function
+ * @return the function result
+ */
+ R apply(T input);
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/IdentityFunction.java b/storm-core/src/jvm/org/apache/storm/streams/operations/IdentityFunction.java
new file mode 100644
index 00000000000..abb9327bef6
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/IdentityFunction.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.operations;
+
+/**
+ * A {@link Function} that returns the input argument itself as the result.
+ *
+ * @param the type of the input
+ */
+public class IdentityFunction implements Function {
+
+ @Override
+ public T apply(T input) {
+ return input;
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/Operation.java b/storm-core/src/jvm/org/apache/storm/streams/operations/Operation.java
new file mode 100644
index 00000000000..77dbe1ed4f4
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/Operation.java
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.operations;
+
+import java.io.Serializable;
+
+/**
+ * The parent interface for any kind of streaming operation.
+ */
+public interface Operation extends Serializable {
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/PairFlatMapFunction.java b/storm-core/src/jvm/org/apache/storm/streams/operations/PairFlatMapFunction.java
new file mode 100644
index 00000000000..376c1ba1176
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/PairFlatMapFunction.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.operations;
+
+import org.apache.storm.streams.Pair;
+
+/**
+ * A function that accepts one argument and returns an {@link Iterable} of {@link Pair} as its result.
+ *
+ * @param the type of the input to the function
+ * @param the key type of the key-value pairs produced as a result
+ * @param the value type of the key-value pairs produced as a result
+ */
+public interface PairFlatMapFunction extends FlatMapFunction> {
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/PairFunction.java b/storm-core/src/jvm/org/apache/storm/streams/operations/PairFunction.java
new file mode 100644
index 00000000000..b153ff18b21
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/PairFunction.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.operations;
+
+import org.apache.storm.streams.Pair;
+
+/**
+ * A function that accepts an argument and produces a key-value Pair.
+ *
+ * @param the type of the input to the function
+ * @param the key type
+ * @param the value type
+ */
+public interface PairFunction extends Function> {
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/PairValueJoiner.java b/storm-core/src/jvm/org/apache/storm/streams/operations/PairValueJoiner.java
new file mode 100644
index 00000000000..ca811018c20
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/PairValueJoiner.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.operations;
+
+import org.apache.storm.streams.Pair;
+
+/**
+ * A {@link ValueJoiner} that joins two values to produce a {@link Pair} of the two values as the result.
+ *
+ * @param the type of the first value
+ * @param the type of the second value
+ */
+public class PairValueJoiner implements ValueJoiner> {
+ /**
+ * Joins two values and produces a {@link Pair} of the values as the result.
+ *
+ * @param value1 the first value
+ * @param value2 the second value
+ * @return a pair of the first and second value
+ */
+ @Override
+ public Pair apply(V1 value1, V2 value2) {
+ return Pair.of(value1, value2);
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/Predicate.java b/storm-core/src/jvm/org/apache/storm/streams/operations/Predicate.java
new file mode 100644
index 00000000000..ae1be2de727
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/Predicate.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.operations;
+
+/**
+ * Represents a predicate (boolean-valued function) of a value.
+ *
+ * @param the value type
+ */
+public interface Predicate extends Operation {
+ /**
+ * Evaluates this predicate on the given argument.
+ *
+ * @param input the input argument
+ * @return true if the input matches the predicate, false otherwise
+ */
+ boolean test(T input);
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/PrintConsumer.java b/storm-core/src/jvm/org/apache/storm/streams/operations/PrintConsumer.java
new file mode 100644
index 00000000000..d301dbfad93
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/PrintConsumer.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.operations;
+
+/**
+ * A consumer that prints the input to stdout.
+ *
+ * @param the value type
+ */
+public class PrintConsumer implements Consumer {
+ @Override
+ public void accept(T input) {
+ System.out.println(input);
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/Reducer.java b/storm-core/src/jvm/org/apache/storm/streams/operations/Reducer.java
new file mode 100644
index 00000000000..04ee70d07c7
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/Reducer.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.operations;
+
+/**
+ * The {@link Reducer} performs an operation on two values of the same type producing a result of the same
+ * type.
+ *
+ * @param the type of the arguments and the result
+ */
+public interface Reducer extends Operation {
+ /**
+ * Applies this function to the given arguments.
+ *
+ * @param arg1 the first argument
+ * @param arg2 the second argument
+ * @return the result
+ */
+ T apply(T arg1, T arg2);
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/ValueJoiner.java b/storm-core/src/jvm/org/apache/storm/streams/operations/ValueJoiner.java
new file mode 100644
index 00000000000..003d097976a
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/ValueJoiner.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.operations;
+
+/**
+ * An interface for joining two values to produce a result.
+ *
+ * @param the type of the first value
+ * @param the type of the second value
+ * @param the result type
+ */
+public interface ValueJoiner extends Operation {
+ /**
+ * Joins two values and produces a result.
+ *
+ * @param value1 the first value
+ * @param value2 the second value
+ * @return the result
+ */
+ R apply(V1 value1, V2 value2);
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Count.java b/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Count.java
new file mode 100644
index 00000000000..fd02d2a0247
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Count.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.operations.aggregators;
+
+import org.apache.storm.streams.operations.Aggregator;
+
+/**
+ * Computes the count of values.
+ *
+ * @param the value type
+ */
+public class Count implements Aggregator {
+ @Override
+ public Long init() {
+ return 0L;
+ }
+
+ @Override
+ public Long apply(T value, Long aggregate) {
+ return aggregate + 1;
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Sum.java b/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Sum.java
new file mode 100644
index 00000000000..e23207562d8
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Sum.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.operations.aggregators;
+
+import org.apache.storm.streams.operations.Aggregator;
+
+/**
+ * Computes the long sum of the input values
+ */
+public class Sum implements Aggregator {
+ @Override
+ public Long init() {
+ return 0L;
+ }
+
+ @Override
+ public Long apply(Number value, Long aggregate) {
+ return value.longValue() + aggregate;
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/PairValueMapper.java b/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/PairValueMapper.java
new file mode 100644
index 00000000000..1490e514df7
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/PairValueMapper.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.operations.mappers;
+
+import org.apache.storm.streams.Pair;
+import org.apache.storm.streams.operations.PairFunction;
+import org.apache.storm.tuple.Tuple;
+
+/**
+ * Extracts a typed key-value pair from a tuple.
+ *
+ * @param the key type
+ * @param the value type
+ */
+public class PairValueMapper implements TupleValueMapper>, PairFunction {
+ private final int keyIndex;
+ private final int valueIndex;
+
+ /**
+ * Constructs a new {@link PairValueMapper} that constructs a pair
+ * from a tuple based on the key and value index.
+ *
+ * @param keyIndex the key index
+ * @param valueIndex the value index
+ */
+ public PairValueMapper(int keyIndex, int valueIndex) {
+ this.keyIndex = keyIndex;
+ this.valueIndex = valueIndex;
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public Pair apply(Tuple input) {
+ return Pair.of((K) input.getValue(keyIndex), (V) input.getValue(valueIndex));
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/TupleValueMapper.java b/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/TupleValueMapper.java
new file mode 100644
index 00000000000..6ffbd886d28
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/TupleValueMapper.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.operations.mappers;
+
+import org.apache.storm.streams.operations.Function;
+import org.apache.storm.tuple.Tuple;
+
+/**
+ * A generic interface for mapping a {@link Tuple} to typed
+ * values.
+ *
+ * @param the result type
+ */
+public interface TupleValueMapper extends Function {
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/ValueMapper.java b/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/ValueMapper.java
new file mode 100644
index 00000000000..0468d0ddce5
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/ValueMapper.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.operations.mappers;
+
+import org.apache.storm.tuple.Tuple;
+
+/**
+ * Extracts a single typed value from a tuple.
+ *
+ * @param the value type
+ */
+public class ValueMapper implements TupleValueMapper {
+ private final int index;
+
+ /**
+ * Constructs a new {@link ValueMapper} that extracts
+ * the value at index 'i' from a tuple.
+ *
+ * @param index the index
+ */
+ public ValueMapper(int index) {
+ this.index = index;
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public T apply(Tuple input) {
+ return (T) input.getValue(index);
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/ValuesMapper.java b/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/ValuesMapper.java
new file mode 100644
index 00000000000..f1d8e3b1b00
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/ValuesMapper.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.operations.mappers;
+
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+
+/**
+ * Constructs a {@link Values} from a {@link Tuple}
+ * based on indicies.
+ */
+public class ValuesMapper implements TupleValueMapper {
+ private final int[] indices;
+
+ /**
+ * Constructs a new {@link ValuesMapper} that extracts
+ * value from a {@link Tuple} at specified indices
+ *
+ * @param indices the indices
+ */
+ public ValuesMapper(int... indices) {
+ this.indices = indices;
+ }
+
+ @Override
+ public Values apply(Tuple input) {
+ Values values = new Values();
+ for (int i : indices) {
+ values.add(input.getValue(i));
+ }
+ return values;
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java
new file mode 100644
index 00000000000..d53a52d4f06
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.processors;
+
+import org.apache.storm.streams.Pair;
+import org.apache.storm.streams.operations.Aggregator;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class AggregateByKeyProcessor extends BaseProcessor> implements BatchProcessor {
+ private final Aggregator aggregator;
+ private final Map state = new HashMap<>();
+
+ public AggregateByKeyProcessor(Aggregator aggregator) {
+ this.aggregator = aggregator;
+ }
+
+ @Override
+ public void execute(Pair input) {
+ K key = input.getFirst();
+ V val = input.getSecond();
+ R agg = state.get(key);
+ if (agg == null) {
+ agg = aggregator.init();
+ }
+ state.put(key, aggregator.apply(val, agg));
+ mayBeForwardAggUpdate(Pair.of(key, state.get(key)));
+ }
+
+ @Override
+ public void finish() {
+ for (Map.Entry entry : state.entrySet()) {
+ context.forward(Pair.of(entry.getKey(), entry.getValue()));
+ }
+ state.clear();
+ }
+
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java
new file mode 100644
index 00000000000..c5a1906202a
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.processors;
+
+import org.apache.storm.streams.operations.Aggregator;
+
+public class AggregateProcessor extends BaseProcessor implements BatchProcessor {
+ private final Aggregator aggregator;
+ private R state;
+
+ public AggregateProcessor(Aggregator aggregator) {
+ this.aggregator = aggregator;
+ }
+
+ @Override
+ public void execute(T input) {
+ if (state == null) {
+ state = aggregator.init();
+ }
+ R curAggregate = (state != null) ? state : aggregator.init();
+ state = aggregator.apply(input, curAggregate);
+ mayBeForwardAggUpdate(state);
+ }
+
+ @Override
+ public void finish() {
+ context.forward(state);
+ state = null;
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/BaseProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/BaseProcessor.java
new file mode 100644
index 00000000000..3ea469c0bab
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/BaseProcessor.java
@@ -0,0 +1,107 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.processors;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.storm.streams.WindowNode.PUNCTUATION;
+
+/**
+ * Base implementation of the {@link Processor} interface that provides
+ * convenience methods {@link #execute(Object)} and {@link #finish()}.
+ *
+ * @param
+ */
+abstract class BaseProcessor implements Processor {
+ private final Set punctuationState = new HashSet<>();
+ protected ProcessorContext context;
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public void init(ProcessorContext context) {
+ this.context = context;
+ }
+
+ /**
+ * {@inheritDoc}
+ * Processors that do not care about the source stream should
+ * override {@link BaseProcessor#execute(Object)} instead.
+ */
+ @Override
+ public void execute(T input, String streamId) {
+ execute(input);
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public void punctuate(String stream) {
+ if ((stream == null) || shouldPunctuate(stream)) {
+ finish();
+ context.forward(PUNCTUATION);
+ punctuationState.clear();
+ }
+ }
+
+ /**
+ * Execute some operation on the input value. Sub classes can override
+ * this when then don't care about the source stream from where the
+ * input is received.
+ *
+ * @param input the input
+ */
+ protected void execute(T input) {
+ // NOOP
+ }
+
+ /**
+ * This is triggered to signal the end of the current
+ * batch of values. Sub classes can override this to
+ * emit the result of a batch of values, for e.g. to emit
+ * the result of an aggregate or join operation on a batch
+ * of values. If a processor does per-value operation like
+ * filter, map etc, they can choose to ignore this.
+ */
+ protected void finish() {
+ // NOOP
+ }
+
+ /**
+ * Forwards the result update to downstream processors. Processors
+ * that operate on a batch of tuples, like aggregation, join etc
+ * can use this to emit the partial results on each input
+ * if they are operating in non-windowed mode.
+ *
+ * @param result the result
+ * @param the result type
+ */
+ protected final void mayBeForwardAggUpdate(R result) {
+ if (!context.isWindowed()) {
+ context.forward(result);
+ }
+ }
+
+ private boolean shouldPunctuate(String parentStream) {
+ punctuationState.add(parentStream);
+ return punctuationState.equals(context.getWindowedParentStreams());
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/BatchProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/BatchProcessor.java
new file mode 100644
index 00000000000..f9a39b42e26
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/BatchProcessor.java
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.processors;
+
+/**
+ * Top level marker interface for processors that computes
+ * results for a batch of tuples like Aggregate, Join etc.
+ */
+public interface BatchProcessor {
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/BranchProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/BranchProcessor.java
new file mode 100644
index 00000000000..f8bc739d4a9
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/BranchProcessor.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.processors;
+
+import org.apache.storm.streams.operations.Predicate;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class BranchProcessor extends BaseProcessor {
+ private final Map, String> predicateToStream = new HashMap<>();
+
+ public void addPredicate(Predicate predicate, String stream) {
+ predicateToStream.put(predicate, stream);
+ }
+
+ @Override
+ public void execute(T input) {
+ for (Map.Entry, String> entry : predicateToStream.entrySet()) {
+ if (entry.getKey().test(input)) {
+ context.forward(input, entry.getValue());
+ break;
+ }
+ }
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ChainedProcessorContext.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ChainedProcessorContext.java
new file mode 100644
index 00000000000..c92d9928bce
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ChainedProcessorContext.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.processors;
+
+import org.apache.storm.streams.ProcessorNode;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * A composite context that holds a chain of {@link ProcessorContext}
+ */
+public class ChainedProcessorContext implements ProcessorContext {
+ private final ProcessorNode processorNode;
+ private final List extends ProcessorContext> contexts;
+
+ public ChainedProcessorContext(ProcessorNode processorNode, List extends ProcessorContext> contexts) {
+ this.processorNode = processorNode;
+ this.contexts = new ArrayList<>(contexts);
+ }
+
+ public ChainedProcessorContext(ProcessorNode processorNode, ProcessorContext... contexts) {
+ this(processorNode, Arrays.asList(contexts));
+ }
+
+ @Override
+ public void forward(T input) {
+ for (ProcessorContext context : contexts) {
+ context.forward(input);
+ }
+ }
+
+ @Override
+ public void forward(T input, String stream) {
+ for (ProcessorContext context : contexts) {
+ context.forward(input, stream);
+ }
+ }
+
+ @Override
+ public boolean isWindowed() {
+ return processorNode.isWindowed();
+ }
+
+ @Override
+ public Set getWindowedParentStreams() {
+ return processorNode.getWindowedParentStreams();
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/EmittingProcessorContext.java b/storm-core/src/jvm/org/apache/storm/streams/processors/EmittingProcessorContext.java
new file mode 100644
index 00000000000..d841080db3b
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/EmittingProcessorContext.java
@@ -0,0 +1,170 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.processors;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Collections2;
+import org.apache.storm.streams.Pair;
+import org.apache.storm.streams.ProcessorNode;
+import org.apache.storm.streams.RefCountedTuple;
+import org.apache.storm.streams.StreamUtil;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.storm.streams.WindowNode.PUNCTUATION;
+
+/**
+ * A context that emits the results to downstream processors which are in
+ * another bolt.
+ */
+public class EmittingProcessorContext implements ProcessorContext {
+ private static final Logger LOG = LoggerFactory.getLogger(EmittingProcessorContext.class);
+ private final ProcessorNode processorNode;
+ private final String outputStreamId;
+ private final OutputCollector collector;
+ private final Fields outputFields;
+ private final Values punctuation;
+ private final List anchors = new ArrayList<>();
+ private boolean emitPunctuation = true;
+ private long eventTimestamp;
+ private String timestampField;
+
+ public EmittingProcessorContext(ProcessorNode processorNode, OutputCollector collector, String outputStreamId) {
+ this.processorNode = processorNode;
+ this.outputStreamId = outputStreamId;
+ this.collector = collector;
+ outputFields = processorNode.getOutputFields();
+ punctuation = createPunctuation();
+ }
+
+ @Override
+ public void forward(T input) {
+ if (input instanceof Pair) {
+ Pair, ?> value = (Pair, ?>) input;
+ emit(new Values(value.getFirst(), value.getSecond()));
+ } else if (PUNCTUATION.equals(input)) {
+ if (emitPunctuation) {
+ emit(punctuation);
+ } else {
+ LOG.debug("Not emitting punctuation since emitPunctuation is false");
+ }
+ maybeAck();
+ } else {
+ emit(new Values(input));
+ }
+ }
+
+ @Override
+ public void forward(T input, String stream) {
+ if (stream.equals(outputStreamId)) {
+ forward(input);
+ }
+ }
+
+ @Override
+ public boolean isWindowed() {
+ return processorNode.isWindowed();
+ }
+
+ @Override
+ public Set getWindowedParentStreams() {
+ return processorNode.getWindowedParentStreams();
+ }
+
+ public void setEmitPunctuation(boolean emitPunctuation) {
+ this.emitPunctuation = emitPunctuation;
+ }
+
+ public void setTimestampField(String fieldName) {
+ timestampField = fieldName;
+ }
+
+ public void setAnchor(RefCountedTuple anchor) {
+ if (processorNode.isWindowed() && processorNode.isBatch()) {
+ anchor.increment();
+ anchors.add(anchor);
+ } else {
+ if (anchors.isEmpty()) {
+ anchors.add(anchor);
+ } else {
+ anchors.set(0, anchor);
+ }
+ /*
+ * track punctuation in non-batch mode so that the
+ * punctuation is acked after all the processors have emitted the punctuation downstream.
+ */
+ if (StreamUtil.isPunctuation(anchor.tuple().getValue(0))) {
+ anchor.increment();
+ }
+ }
+ }
+
+ public void setEventTimestamp(long timestamp) {
+ this.eventTimestamp = timestamp;
+ }
+
+ private Values createPunctuation() {
+ Values values = new Values();
+ for (int i = 0; i < outputFields.size(); i++) {
+ values.add(PUNCTUATION);
+ }
+ return values;
+ }
+
+ private void maybeAck() {
+ if (!anchors.isEmpty()) {
+ for (RefCountedTuple anchor : anchors) {
+ anchor.decrement();
+ if (anchor.shouldAck()) {
+ LOG.debug("Acking {} ", anchor);
+ collector.ack(anchor.tuple());
+ anchor.setAcked();
+ }
+ }
+ anchors.clear();
+ }
+ }
+
+ private Collection tuples(Collection anchors) {
+ return anchors.stream().map(RefCountedTuple::tuple).collect(Collectors.toList());
+ }
+
+ private void emit(Values values) {
+ if (timestampField != null) {
+ values.add(eventTimestamp);
+ }
+ if (anchors.isEmpty()) {
+ // for windowed bolt, windowed output collector will do the anchoring/acking
+ LOG.debug("Emit un-anchored, outputStreamId: {}, values: {}", outputStreamId, values);
+ collector.emit(outputStreamId, values);
+ } else {
+ LOG.debug("Emit, outputStreamId: {}, anchors: {}, values: {}", outputStreamId, anchors, values);
+ collector.emit(outputStreamId, tuples(anchors), values);
+ }
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/FilterProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/FilterProcessor.java
new file mode 100644
index 00000000000..f245a445b46
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/FilterProcessor.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.processors;
+
+import org.apache.storm.streams.operations.Predicate;
+
+public class FilterProcessor extends BaseProcessor {
+ private final Predicate predicate;
+
+ public FilterProcessor(Predicate predicate) {
+ this.predicate = predicate;
+ }
+
+ @Override
+ public void execute(T input) {
+ if (predicate.test(input)) {
+ context.forward(input);
+ }
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/FlatMapProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/FlatMapProcessor.java
new file mode 100644
index 00000000000..fdc86fee130
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/FlatMapProcessor.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.processors;
+
+import org.apache.storm.streams.operations.FlatMapFunction;
+
+public class FlatMapProcessor extends BaseProcessor {
+ private final FlatMapFunction function;
+
+ public FlatMapProcessor(FlatMapFunction function) {
+ this.function = function;
+ }
+
+ @Override
+ public void execute(T input) {
+ for (R res : function.apply(input)) {
+ context.forward(res);
+ }
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/FlatMapValuesProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/FlatMapValuesProcessor.java
new file mode 100644
index 00000000000..92953a34602
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/FlatMapValuesProcessor.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.processors;
+
+import org.apache.storm.streams.Pair;
+import org.apache.storm.streams.operations.FlatMapFunction;
+
+public class FlatMapValuesProcessor extends BaseProcessor> {
+ private final FlatMapFunction function;
+
+ public FlatMapValuesProcessor(FlatMapFunction function) {
+ this.function = function;
+ }
+
+ @Override
+ protected void execute(Pair input) {
+ for (R res : function.apply(input.getSecond())) {
+ context.forward(Pair.of(input.getFirst(), res));
+ }
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ForEachProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ForEachProcessor.java
new file mode 100644
index 00000000000..323b53288c4
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ForEachProcessor.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.processors;
+
+import org.apache.storm.streams.operations.Consumer;
+
+public class ForEachProcessor extends BaseProcessor {
+ private final Consumer consumer;
+
+ public ForEachProcessor(Consumer consumer) {
+ this.consumer = consumer;
+ }
+
+ @Override
+ public void execute(T input) {
+ consumer.accept(input);
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ForwardingProcessorContext.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ForwardingProcessorContext.java
new file mode 100644
index 00000000000..49eb836391f
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ForwardingProcessorContext.java
@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.processors;
+
+import com.google.common.collect.Multimap;
+import org.apache.storm.streams.ProcessorNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Set;
+
+import static org.apache.storm.streams.WindowNode.PUNCTUATION;
+
+/**
+ * A context that emits the results to downstream processors which are in
+ * the same bolt.
+ */
+public class ForwardingProcessorContext implements ProcessorContext {
+ private static final Logger LOG = LoggerFactory.getLogger(ForwardingProcessorContext.class);
+ private final ProcessorNode processorNode;
+ private final Multimap streamToChildren;
+ private final Set streams;
+
+ public ForwardingProcessorContext(ProcessorNode processorNode, Multimap streamToChildren) {
+ this.processorNode = processorNode;
+ this.streamToChildren = streamToChildren;
+ this.streams = streamToChildren.keySet();
+ }
+
+ @Override
+ public void forward(T input) {
+ if (PUNCTUATION.equals(input)) {
+ finishAllStreams();
+ } else {
+ executeAllStreams(input);
+ }
+ }
+
+ @Override
+ public void forward(T input, String stream) {
+ if (PUNCTUATION.equals(input)) {
+ finish(stream);
+ } else {
+ execute(input, stream);
+ }
+ }
+
+ @Override
+ public boolean isWindowed() {
+ return processorNode.isWindowed();
+ }
+
+ @Override
+ public Set getWindowedParentStreams() {
+ return processorNode.getWindowedParentStreams();
+ }
+
+ private void finishAllStreams() {
+ for (String stream : streams) {
+ finish(stream);
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ private void finish(String stream) {
+ for (ProcessorNode node : streamToChildren.get(stream)) {
+ LOG.debug("Punctuating processor: {}", node);
+ Processor processor = (Processor) node.getProcessor();
+ processor.punctuate(stream);
+ }
+ }
+
+ private void executeAllStreams(T input) {
+ for (String stream : streams) {
+ execute(input, stream);
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ private void execute(T input, String stream) {
+ for (ProcessorNode node : streamToChildren.get(stream)) {
+ LOG.debug("Forward input: {} to processor node: {}", input, node);
+ Processor processor = (Processor) node.getProcessor();
+ processor.execute(input, stream);
+ }
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/JoinProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/JoinProcessor.java
new file mode 100644
index 00000000000..d56cfea9ad1
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/JoinProcessor.java
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.processors;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.apache.storm.streams.Pair;
+import org.apache.storm.streams.Tuple3;
+import org.apache.storm.streams.operations.ValueJoiner;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Provides equi-join implementation based on simple hash-join.
+ */
+public class JoinProcessor extends BaseProcessor> implements BatchProcessor {
+ private final ValueJoiner valueJoiner;
+ private final String leftStream;
+ private final String rightStream;
+ private final List> leftRows = new ArrayList<>();
+ private final List> rightRows = new ArrayList<>();
+
+ public JoinProcessor(String leftStream, String rightStream, ValueJoiner valueJoiner) {
+ this.valueJoiner = valueJoiner;
+ this.leftStream = leftStream;
+ this.rightStream = rightStream;
+ }
+
+ @Override
+ public void execute(Pair input, String sourceStream) {
+ K key = input.getFirst();
+ if (sourceStream.equals(leftStream)) {
+ V1 val = (V1) input.getSecond();
+ Pair pair = Pair.of(key, val);
+ leftRows.add(pair);
+ if (!context.isWindowed()) {
+ joinAndForward(Collections.singletonList(pair), rightRows);
+ }
+ } else if (sourceStream.equals(rightStream)) {
+ V2 val = (V2) input.getSecond();
+ Pair pair = Pair.of(key, val);
+ rightRows.add(pair);
+ if (!context.isWindowed()) {
+ joinAndForward(leftRows, Collections.singletonList(pair));
+ }
+ }
+ }
+
+ @Override
+ public void finish() {
+ joinAndForward(leftRows, rightRows);
+ leftRows.clear();
+ rightRows.clear();
+ }
+
+ public String getLeftStream() {
+ return leftStream;
+ }
+
+ public String getRightStream() {
+ return rightStream;
+ }
+
+ private void joinAndForward(List> leftRows, List> rightRows) {
+ if (leftRows.size() <= rightRows.size()) {
+ for (Tuple3 res : join(getJoinTable(leftRows), rightRows)) {
+ context.forward(Pair.of(res._1, valueJoiner.apply(res._2, res._3)));
+ }
+ } else {
+ for (Tuple3 res : join(getJoinTable(rightRows), leftRows)) {
+ context.forward(Pair.of(res._1, valueJoiner.apply(res._3, res._2)));
+ }
+ }
+ }
+
+ private List> join(Multimap tab, List> rows) {
+ List> res = new ArrayList<>();
+ for (Pair row : rows) {
+ for (T1 mapValue : tab.get(row.getFirst())) {
+ if (mapValue != null) {
+ res.add(new Tuple3<>(row.getFirst(), mapValue, row.getSecond()));
+ }
+ }
+ }
+ return res;
+ }
+
+ private Multimap getJoinTable(List> rows) {
+ Multimap m = ArrayListMultimap.create();
+ for (Pair v : rows) {
+ m.put(v.getFirst(), v.getSecond());
+ }
+ return m;
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/MapProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/MapProcessor.java
new file mode 100644
index 00000000000..e9f89a9fc60
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/MapProcessor.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.processors;
+
+import org.apache.storm.streams.operations.Function;
+
+public class MapProcessor extends BaseProcessor {
+ private final Function function;
+
+ public MapProcessor(Function function) {
+ this.function = function;
+ }
+
+ @Override
+ public void execute(T input) {
+ context.forward(function.apply(input));
+ }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/MapValuesProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/MapValuesProcessor.java
new file mode 100644
index 00000000000..a963e2e48e2
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/MapValuesProcessor.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.streams.processors;
+
+import org.apache.storm.streams.Pair;
+import org.apache.storm.streams.operations.Function;
+
+public class MapValuesProcessor