You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by ka...@apache.org on 2017/02/07 01:28:14 UTC

[01/12] storm git commit: [STORM-1961] Stream api for storm core use cases

Repository: storm
Updated Branches:
  refs/heads/master a7cdfefd7 -> b5f02d4e4


http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/topology/StatefulWindowedBoltExecutor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/topology/StatefulWindowedBoltExecutor.java b/storm-core/src/jvm/org/apache/storm/topology/StatefulWindowedBoltExecutor.java
index aff46e6..2fc6f8d 100644
--- a/storm-core/src/jvm/org/apache/storm/topology/StatefulWindowedBoltExecutor.java
+++ b/storm-core/src/jvm/org/apache/storm/topology/StatefulWindowedBoltExecutor.java
@@ -207,14 +207,14 @@ public class StatefulWindowedBoltExecutor<T extends State> extends WindowedBoltE
             }
 
             @Override
-            public void onActivation(List<Tuple> events, List<Tuple> newEvents, List<Tuple> expired) {
+            public void onActivation(List<Tuple> events, List<Tuple> newEvents, List<Tuple> expired, Long timestamp) {
                 if (isRecovering()) {
                     String msg = String.format("Unexpected activation with events %s, newEvents %s, expired %s in recovering state. " +
                                                        "recoveryStates %s ", events, newEvents, expired, recoveryStates);
                     LOG.error(msg);
                     throw new IllegalStateException(msg);
                 } else {
-                    parentListener.onActivation(events, newEvents, expired);
+                    parentListener.onActivation(events, newEvents, expired, timestamp);
                     updateWindowState(expired, newEvents);
                 }
             }

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java b/storm-core/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java
index fd98274..b592e0b 100644
--- a/storm-core/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java
+++ b/storm-core/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java
@@ -327,9 +327,9 @@ public class WindowedBoltExecutor implements IRichBolt {
             }
 
             @Override
-            public void onActivation(List<Tuple> tuples, List<Tuple> newTuples, List<Tuple> expiredTuples) {
+            public void onActivation(List<Tuple> tuples, List<Tuple> newTuples, List<Tuple> expiredTuples, Long timestamp) {
                 windowedOutputCollector.setContext(tuples);
-                bolt.execute(new TupleWindowImpl(tuples, newTuples, expiredTuples));
+                bolt.execute(new TupleWindowImpl(tuples, newTuples, expiredTuples, timestamp));
             }
         };
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/topology/base/BaseWindowedBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/topology/base/BaseWindowedBolt.java b/storm-core/src/jvm/org/apache/storm/topology/base/BaseWindowedBolt.java
index 2e2af0d..ba97c26 100644
--- a/storm-core/src/jvm/org/apache/storm/topology/base/BaseWindowedBolt.java
+++ b/storm-core/src/jvm/org/apache/storm/topology/base/BaseWindowedBolt.java
@@ -27,6 +27,7 @@ import org.apache.storm.windowing.TimestampExtractor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.Serializable;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
@@ -40,7 +41,7 @@ public abstract class BaseWindowedBolt implements IWindowedBolt {
     /**
      * Holds a count value for count based windows and sliding intervals.
      */
-    public static class Count {
+    public static class Count implements Serializable {
         public final int value;
 
         public Count(int value) {
@@ -58,6 +59,22 @@ public abstract class BaseWindowedBolt implements IWindowedBolt {
         }
 
         @Override
+        public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            Count count = (Count) o;
+
+            return value == count.value;
+
+        }
+
+        @Override
+        public int hashCode() {
+            return value;
+        }
+
+        @Override
         public String toString() {
             return "Count{" +
                     "value=" + value +
@@ -68,7 +85,7 @@ public abstract class BaseWindowedBolt implements IWindowedBolt {
     /**
      * Holds a Time duration for time based windows and sliding intervals.
      */
-    public static class Duration {
+    public static class Duration implements Serializable {
         public final int value;
 
         public Duration(int value, TimeUnit timeUnit) {
@@ -126,6 +143,22 @@ public abstract class BaseWindowedBolt implements IWindowedBolt {
         }
 
         @Override
+        public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            Duration duration = (Duration) o;
+
+            return value == duration.value;
+
+        }
+
+        @Override
+        public int hashCode() {
+            return value;
+        }
+
+        @Override
         public String toString() {
             return "Duration{" +
                     "value=" + value +

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/trident/windowing/AbstractTridentWindowManager.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/trident/windowing/AbstractTridentWindowManager.java b/storm-core/src/jvm/org/apache/storm/trident/windowing/AbstractTridentWindowManager.java
index f93527a..a8fbb41 100644
--- a/storm-core/src/jvm/org/apache/storm/trident/windowing/AbstractTridentWindowManager.java
+++ b/storm-core/src/jvm/org/apache/storm/trident/windowing/AbstractTridentWindowManager.java
@@ -121,7 +121,7 @@ public abstract class AbstractTridentWindowManager<T> implements ITridentWindowM
         }
 
         @Override
-        public void onActivation(List<T> events, List<T> newEvents, List<T> expired) {
+        public void onActivation(List<T> events, List<T> newEvents, List<T> expired, Long timestamp) {
             LOG.debug("onActivation is invoked with events size: [{}]", events.size());
             // trigger occurred, create an aggregation and keep them in store
             int currentTriggerId = triggerId.incrementAndGet();

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/windowing/CountEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/windowing/CountEvictionPolicy.java b/storm-core/src/jvm/org/apache/storm/windowing/CountEvictionPolicy.java
index fb12202..6a9a4f8 100644
--- a/storm-core/src/jvm/org/apache/storm/windowing/CountEvictionPolicy.java
+++ b/storm-core/src/jvm/org/apache/storm/windowing/CountEvictionPolicy.java
@@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicLong;
 public class CountEvictionPolicy<T> implements EvictionPolicy<T> {
     protected final int threshold;
     protected final AtomicLong currentCount;
+    private EvictionContext context;
 
     public CountEvictionPolicy(int count) {
         this.threshold = count;
@@ -62,7 +63,12 @@ public class CountEvictionPolicy<T> implements EvictionPolicy<T> {
 
     @Override
     public void setContext(EvictionContext context) {
-        // NOOP
+        this.context = context;
+    }
+
+    @Override
+    public EvictionContext getContext() {
+        return context;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/windowing/EvictionPolicy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/windowing/EvictionPolicy.java b/storm-core/src/jvm/org/apache/storm/windowing/EvictionPolicy.java
index 05e4d93..774d0a3 100644
--- a/storm-core/src/jvm/org/apache/storm/windowing/EvictionPolicy.java
+++ b/storm-core/src/jvm/org/apache/storm/windowing/EvictionPolicy.java
@@ -72,4 +72,11 @@ public interface EvictionPolicy<T> {
      */
     void setContext(EvictionContext context);
 
+    /**
+     * Returns the current context that is part of this eviction policy
+     *
+     * @return the eviction context
+     */
+    EvictionContext getContext();
+
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/windowing/TimeEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/windowing/TimeEvictionPolicy.java b/storm-core/src/jvm/org/apache/storm/windowing/TimeEvictionPolicy.java
index e646207..802e6bb 100644
--- a/storm-core/src/jvm/org/apache/storm/windowing/TimeEvictionPolicy.java
+++ b/storm-core/src/jvm/org/apache/storm/windowing/TimeEvictionPolicy.java
@@ -22,11 +22,7 @@ package org.apache.storm.windowing;
  */
 public class TimeEvictionPolicy<T> implements EvictionPolicy<T> {
     private final int windowLength;
-    /**
-     * The reference time in millis for window calculations and
-     * expiring events. If not set it will default to System.currentTimeMillis()
-     */
-    protected Long referenceTime;
+    protected EvictionContext evictionContext;
 
     /**
      * Constructs a TimeEvictionPolicy that evicts events older
@@ -43,7 +39,7 @@ public class TimeEvictionPolicy<T> implements EvictionPolicy<T> {
      */
     @Override
     public Action evict(Event<T> event) {
-        long now = referenceTime == null ? System.currentTimeMillis() : referenceTime;
+        long now = evictionContext == null ? System.currentTimeMillis() : evictionContext.getReferenceTime();
         long diff = now - event.getTimestamp();
         if (diff >= windowLength) {
             return Action.EXPIRE;
@@ -58,14 +54,19 @@ public class TimeEvictionPolicy<T> implements EvictionPolicy<T> {
 
     @Override
     public void setContext(EvictionContext context) {
-        referenceTime = context.getReferenceTime();
+        this.evictionContext = context;
+    }
+
+    @Override
+    public EvictionContext getContext() {
+        return evictionContext;
     }
 
     @Override
     public String toString() {
         return "TimeEvictionPolicy{" +
                 "windowLength=" + windowLength +
-                ", referenceTime=" + referenceTime +
+                ", evictionContext=" + evictionContext +
                 '}';
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/windowing/TupleWindowImpl.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/windowing/TupleWindowImpl.java b/storm-core/src/jvm/org/apache/storm/windowing/TupleWindowImpl.java
index 2560d25..1e8b022 100644
--- a/storm-core/src/jvm/org/apache/storm/windowing/TupleWindowImpl.java
+++ b/storm-core/src/jvm/org/apache/storm/windowing/TupleWindowImpl.java
@@ -28,11 +28,17 @@ public class TupleWindowImpl implements TupleWindow {
     private final List<Tuple> tuples;
     private final List<Tuple> newTuples;
     private final List<Tuple> expiredTuples;
+    private final Long timestamp;
 
     public TupleWindowImpl(List<Tuple> tuples, List<Tuple> newTuples, List<Tuple> expiredTuples) {
+        this(tuples, newTuples, expiredTuples, null);
+    }
+
+    public TupleWindowImpl(List<Tuple> tuples, List<Tuple> newTuples, List<Tuple> expiredTuples, Long timestamp) {
         this.tuples = tuples;
         this.newTuples = newTuples;
         this.expiredTuples = expiredTuples;
+        this.timestamp = timestamp;
     }
 
     @Override
@@ -51,6 +57,11 @@ public class TupleWindowImpl implements TupleWindow {
     }
 
     @Override
+    public Long getTimestamp() {
+        return timestamp != null ? timestamp : System.currentTimeMillis();
+    }
+
+    @Override
     public String toString() {
         return "TupleWindowImpl{" +
                 "tuples=" + tuples +

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/windowing/WatermarkCountEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/windowing/WatermarkCountEvictionPolicy.java b/storm-core/src/jvm/org/apache/storm/windowing/WatermarkCountEvictionPolicy.java
index 74240bb..7304366 100644
--- a/storm-core/src/jvm/org/apache/storm/windowing/WatermarkCountEvictionPolicy.java
+++ b/storm-core/src/jvm/org/apache/storm/windowing/WatermarkCountEvictionPolicy.java
@@ -24,11 +24,6 @@ package org.apache.storm.windowing;
  * @param <T> the type of event tracked by this policy.
  */
 public class WatermarkCountEvictionPolicy<T> extends CountEvictionPolicy<T> {
-    /*
-     * The reference time in millis for window calculations and
-     * expiring events. If not set it will default to System.currentTimeMillis()
-     */
-    private long referenceTime;
     private long processed = 0L;
 
     public WatermarkCountEvictionPolicy(int count) {
@@ -38,7 +33,7 @@ public class WatermarkCountEvictionPolicy<T> extends CountEvictionPolicy<T> {
     @Override
     public Action evict(Event<T> event) {
         Action action;
-        if (event.getTimestamp() <= referenceTime && processed < currentCount.get()) {
+        if (event.getTimestamp() <= super.getContext().getReferenceTime() && processed < currentCount.get()) {
             action = super.evict(event);
             if (action == Action.PROCESS) {
                 ++processed;
@@ -56,7 +51,7 @@ public class WatermarkCountEvictionPolicy<T> extends CountEvictionPolicy<T> {
 
     @Override
     public void setContext(EvictionContext context) {
-        referenceTime = context.getReferenceTime();
+        super.setContext(context);
         if (context.getCurrentCount() != null) {
             currentCount.set(context.getCurrentCount());
         } else {
@@ -68,7 +63,6 @@ public class WatermarkCountEvictionPolicy<T> extends CountEvictionPolicy<T> {
     @Override
     public String toString() {
         return "WatermarkCountEvictionPolicy{" +
-                "referenceTime=" + referenceTime +
                 "} " + super.toString();
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/windowing/WatermarkTimeEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/windowing/WatermarkTimeEvictionPolicy.java b/storm-core/src/jvm/org/apache/storm/windowing/WatermarkTimeEvictionPolicy.java
index 53361d2..e5ecba4 100644
--- a/storm-core/src/jvm/org/apache/storm/windowing/WatermarkTimeEvictionPolicy.java
+++ b/storm-core/src/jvm/org/apache/storm/windowing/WatermarkTimeEvictionPolicy.java
@@ -45,7 +45,6 @@ public class WatermarkTimeEvictionPolicy<T> extends TimeEvictionPolicy<T> {
      */
     public WatermarkTimeEvictionPolicy(int windowLength, int lag) {
         super(windowLength);
-        referenceTime = 0L;
         this.lag = lag;
     }
 
@@ -58,7 +57,8 @@ public class WatermarkTimeEvictionPolicy<T> extends TimeEvictionPolicy<T> {
      */
     @Override
     public Action evict(Event<T> event) {
-        long diff = referenceTime - event.getTimestamp();
+        long referenceTime = evictionContext.getReferenceTime() != null ? evictionContext.getReferenceTime() : 0L;
+        long diff =  referenceTime - event.getTimestamp();
         if (diff < -lag) {
             return Action.STOP;
         } else if (diff < 0) {

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/windowing/Window.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/windowing/Window.java b/storm-core/src/jvm/org/apache/storm/windowing/Window.java
index 8382448..9a62eef 100644
--- a/storm-core/src/jvm/org/apache/storm/windowing/Window.java
+++ b/storm-core/src/jvm/org/apache/storm/windowing/Window.java
@@ -45,4 +45,11 @@ public interface Window<T> {
      * @return the list of events expired from the window.
      */
     List<T> getExpired();
+
+    /**
+     * If processing based on event time, returns the watermark time otherwise the current timestamp.
+     *
+     * @return the window timestamp
+     */
+    Long getTimestamp();
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/windowing/WindowLifecycleListener.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/windowing/WindowLifecycleListener.java b/storm-core/src/jvm/org/apache/storm/windowing/WindowLifecycleListener.java
index 03c0213..ea2c997 100644
--- a/storm-core/src/jvm/org/apache/storm/windowing/WindowLifecycleListener.java
+++ b/storm-core/src/jvm/org/apache/storm/windowing/WindowLifecycleListener.java
@@ -37,6 +37,7 @@ public interface WindowLifecycleListener<T> {
      * @param events the list of current events in the window.
      * @param newEvents the newly added events since last activation.
      * @param expired the expired events since last activation.
+     * @param referenceTime the reference (event or processing) time that resulted in activation
      */
-    void onActivation(List<T> events, List<T> newEvents, List<T> expired);
+    void onActivation(List<T> events, List<T> newEvents, List<T> expired, Long referenceTime);
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/windowing/WindowManager.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/windowing/WindowManager.java b/storm-core/src/jvm/org/apache/storm/windowing/WindowManager.java
index 792509e..8021ba8 100644
--- a/storm-core/src/jvm/org/apache/storm/windowing/WindowManager.java
+++ b/storm-core/src/jvm/org/apache/storm/windowing/WindowManager.java
@@ -142,7 +142,7 @@ public class WindowManager<T> implements TriggerHandler {
         if (!events.isEmpty()) {
             prevWindowEvents.addAll(windowEvents);
             LOG.debug("invoking windowLifecycleListener onActivation, [{}] events in window.", events.size());
-            windowLifecycleListener.onActivation(events, newEvents, expired);
+            windowLifecycleListener.onActivation(events, newEvents, expired, evictionPolicy.getContext().getReferenceTime());
         } else {
             LOG.debug("No events in the window, skipping onActivation");
         }

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java b/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java
new file mode 100644
index 0000000..e9d5127
--- /dev/null
+++ b/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java
@@ -0,0 +1,165 @@
+/**
+ * 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.operations.aggregators.Sum;
+import org.apache.storm.streams.processors.AggregateProcessor;
+import org.apache.storm.streams.processors.FilterProcessor;
+import org.apache.storm.streams.processors.Processor;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+import org.jgrapht.DirectedGraph;
+import org.jgrapht.graph.DefaultDirectedGraph;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Set;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Unit tests for {@link ProcessorBolt}
+ */
+public class ProcessorBoltTest {
+    TopologyContext mockTopologyContext;
+    OutputCollector mockOutputCollector;
+    ProcessorBolt bolt;
+    Tuple mockTuple1;
+    Tuple mockTuple2;
+    Tuple mockTuple3;
+    Tuple punctuation;
+    Multimap<String, ProcessorNode> mockStreamToProcessors;
+    DirectedGraph<Node, Edge> graph;
+
+    @Before
+    public void setUp() throws Exception {
+        mockTopologyContext = Mockito.mock(TopologyContext.class);
+        mockOutputCollector = Mockito.mock(OutputCollector.class);
+        mockTuple1 = Mockito.mock(Tuple.class);
+        mockTuple2 = Mockito.mock(Tuple.class);
+        mockTuple3 = Mockito.mock(Tuple.class);
+        setUpMockTuples(mockTuple1, mockTuple2, mockTuple3);
+        punctuation = Mockito.mock(Tuple.class);
+        setUpPunctuation(punctuation);
+        mockStreamToProcessors = Mockito.mock(Multimap.class);
+        graph = new DefaultDirectedGraph(new StreamsEdgeFactory());
+
+    }
+
+    @Test
+    public void testEmitAndAck() throws Exception {
+        setUpProcessorBolt(new FilterProcessor<Integer>(x -> true));
+        bolt.execute(mockTuple1);
+        ArgumentCaptor<Collection> anchor = ArgumentCaptor.forClass(Collection.class);
+        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
+        ArgumentCaptor<String> os = ArgumentCaptor.forClass(String.class);
+        Mockito.verify(mockOutputCollector).emit(os.capture(), anchor.capture(), values.capture());
+        assertEquals("outputstream", os.getValue());
+        assertArrayEquals(new Object[]{mockTuple1}, anchor.getValue().toArray());
+        assertEquals(new Values(100), values.getValue());
+        Mockito.verify(mockOutputCollector, Mockito.times(1)).ack(mockTuple1);
+    }
+
+    @Test
+    public void testAggResultAndAck() throws Exception {
+        setUpProcessorBolt(new AggregateProcessor<>(new Sum()), Collections.singleton("inputstream"), true, null);
+        bolt.execute(mockTuple2);
+        bolt.execute(mockTuple3);
+        bolt.execute(punctuation);
+        ArgumentCaptor<Collection> anchor = ArgumentCaptor.forClass(Collection.class);
+        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
+        ArgumentCaptor<String> os = ArgumentCaptor.forClass(String.class);
+        Mockito.verify(mockOutputCollector, Mockito.times(2)).emit(os.capture(), anchor.capture(), values.capture());
+        assertArrayEquals(new Object[]{mockTuple2, mockTuple3, punctuation}, anchor.getAllValues().get(0).toArray());
+        assertArrayEquals(new Object[]{mockTuple2, mockTuple3, punctuation}, anchor.getAllValues().get(1).toArray());
+        assertArrayEquals(new Object[]{new Values(200L), new Values("__punctuation")}, values.getAllValues().toArray());
+        assertArrayEquals(new Object[]{"outputstream", "outputstream"}, os.getAllValues().toArray());
+        Mockito.verify(mockOutputCollector).ack(mockTuple2);
+        Mockito.verify(mockOutputCollector).ack(mockTuple3);
+        Mockito.verify(mockOutputCollector).ack(punctuation);
+    }
+
+    @Test
+    public void testEmitTs() throws Exception {
+        Tuple tupleWithTs = Mockito.mock(Tuple.class);
+        setUpMockTuples(tupleWithTs);
+        Mockito.when(tupleWithTs.getLongByField("ts")).thenReturn(12345L);
+        setUpProcessorBolt(new FilterProcessor(x -> true), "ts");
+        bolt.execute(tupleWithTs);
+        ArgumentCaptor<Collection> anchor = ArgumentCaptor.forClass(Collection.class);
+        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
+        ArgumentCaptor<String> os = ArgumentCaptor.forClass(String.class);
+        Mockito.verify(mockOutputCollector).emit(os.capture(), anchor.capture(), values.capture());
+        assertEquals("outputstream", os.getValue());
+        assertArrayEquals(new Object[]{tupleWithTs}, anchor.getValue().toArray());
+        assertEquals(new Values(100, 12345L), values.getValue());
+        Mockito.verify(mockOutputCollector, Mockito.times(1)).ack(tupleWithTs);
+    }
+
+    private void setUpProcessorBolt(Processor<?> processor) {
+        setUpProcessorBolt(processor, Collections.emptySet(), false, null);
+    }
+
+    private void setUpProcessorBolt(Processor<?> processor, String tsFieldName) {
+        setUpProcessorBolt(processor, Collections.emptySet(), false, tsFieldName);
+    }
+
+    private void setUpProcessorBolt(Processor<?> processor,
+                                    Set<String> windowedParentStreams,
+                                    boolean isWindowed,
+                                    String tsFieldName) {
+        ProcessorNode node = new ProcessorNode(processor, "outputstream", new Fields("value"));
+        node.setWindowedParentStreams(windowedParentStreams);
+        node.setWindowed(isWindowed);
+        Mockito.when(mockStreamToProcessors.get(Mockito.anyString())).thenReturn(Collections.singletonList(node));
+        graph.addVertex(node);
+        bolt = new ProcessorBolt("bolt1", graph, Collections.singletonList(node));
+        if (tsFieldName != null && !tsFieldName.isEmpty()) {
+            bolt.setTimestampField(tsFieldName);
+        }
+        bolt.setStreamToInitialProcessors(mockStreamToProcessors);
+        bolt.prepare(new HashMap<>(), mockTopologyContext, mockOutputCollector);
+    }
+
+    private void setUpMockTuples(Tuple... tuples) {
+        for (Tuple tuple : tuples) {
+            Mockito.when(tuple.size()).thenReturn(1);
+            Mockito.when(tuple.getValue(0)).thenReturn(100);
+            Mockito.when(tuple.getSourceComponent()).thenReturn("bolt0");
+            Mockito.when(tuple.getSourceStreamId()).thenReturn("inputstream");
+        }
+    }
+
+    private void setUpPunctuation(Tuple punctuation) {
+        Mockito.when(punctuation.size()).thenReturn(1);
+        Mockito.when(punctuation.getValue(0)).thenReturn(WindowNode.PUNCTUATION);
+        Mockito.when(punctuation.getSourceComponent()).thenReturn("bolt0");
+        Mockito.when(punctuation.getSourceStreamId()).thenReturn("inputstream");
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/test/jvm/org/apache/storm/streams/StatefulProcessorBoltTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/streams/StatefulProcessorBoltTest.java b/storm-core/test/jvm/org/apache/storm/streams/StatefulProcessorBoltTest.java
new file mode 100644
index 0000000..dbc7e27
--- /dev/null
+++ b/storm-core/test/jvm/org/apache/storm/streams/StatefulProcessorBoltTest.java
@@ -0,0 +1,100 @@
+/**
+ * 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.state.KeyValueState;
+import org.apache.storm.streams.operations.aggregators.Count;
+import org.apache.storm.streams.processors.Processor;
+import org.apache.storm.streams.processors.UpdateStateByKeyProcessor;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+import org.jgrapht.DirectedGraph;
+import org.jgrapht.graph.DefaultDirectedGraph;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Unit tests for {@link StatefulProcessorBolt}
+ */
+public class StatefulProcessorBoltTest {
+    TopologyContext mockTopologyContext;
+    OutputCollector mockOutputCollector;
+    StatefulProcessorBolt<String, Long> bolt;
+    Tuple mockTuple1;
+    DirectedGraph<Node, Edge> graph;
+    Multimap<String, ProcessorNode> mockStreamToProcessors;
+    KeyValueState<String, Long> mockKeyValueState;
+
+    @Before
+    public void setUp() throws Exception {
+        mockTopologyContext = Mockito.mock(TopologyContext.class);
+        mockOutputCollector = Mockito.mock(OutputCollector.class);
+        mockTuple1 = Mockito.mock(Tuple.class);
+        mockStreamToProcessors = Mockito.mock(Multimap.class);
+        mockKeyValueState = Mockito.mock(KeyValueState.class);
+        setUpMockTuples(mockTuple1);
+    }
+
+    @Test
+    public void testEmitAndAck() throws Exception {
+        setUpStatefulProcessorBolt(new UpdateStateByKeyProcessor<>(new Count<>()));
+        bolt.execute(mockTuple1);
+        ArgumentCaptor<Collection> anchor = ArgumentCaptor.forClass(Collection.class);
+        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
+        ArgumentCaptor<String> os = ArgumentCaptor.forClass(String.class);
+        Mockito.verify(mockOutputCollector).emit(os.capture(), anchor.capture(), values.capture());
+        assertEquals("outputstream", os.getValue());
+        assertArrayEquals(new Object[]{mockTuple1}, anchor.getValue().toArray());
+        assertEquals(new Values("k", 1L), values.getValue());
+        Mockito.verify(mockOutputCollector, Mockito.times(1)).ack(mockTuple1);
+        Mockito.verify(mockKeyValueState, Mockito.times(1)).put("k", 1L );
+    }
+
+    private void setUpStatefulProcessorBolt(Processor<?> processor) {
+        ProcessorNode node = new ProcessorNode(processor, "outputstream", new Fields("value"));
+        Mockito.when(mockStreamToProcessors.get(Mockito.anyString())).thenReturn(Collections.singletonList(node));
+        graph = new DefaultDirectedGraph(new StreamsEdgeFactory());
+        graph.addVertex(node);
+        bolt = new StatefulProcessorBolt<>("bolt1", graph, Collections.singletonList(node));
+        bolt.setStreamToInitialProcessors(mockStreamToProcessors);
+        bolt.prepare(new HashMap<>(), mockTopologyContext, mockOutputCollector);
+        bolt.initState(mockKeyValueState);
+    }
+
+    private void setUpMockTuples(Tuple... tuples) {
+        for (Tuple tuple : tuples) {
+            Mockito.when(tuple.size()).thenReturn(1);
+            Mockito.when(tuple.getValue(0)).thenReturn(Pair.of("k", "v"));
+            Mockito.when(tuple.getSourceComponent()).thenReturn("bolt0");
+            Mockito.when(tuple.getSourceStreamId()).thenReturn("inputstream");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/test/jvm/org/apache/storm/streams/StreamBuilderTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/streams/StreamBuilderTest.java b/storm-core/test/jvm/org/apache/storm/streams/StreamBuilderTest.java
new file mode 100644
index 0000000..1498ae4
--- /dev/null
+++ b/storm-core/test/jvm/org/apache/storm/streams/StreamBuilderTest.java
@@ -0,0 +1,219 @@
+/**
+ * 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.generated.Bolt;
+import org.apache.storm.generated.GlobalStreamId;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.NullStruct;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.streams.operations.aggregators.Count;
+import org.apache.storm.streams.operations.mappers.PairValueMapper;
+import org.apache.storm.streams.operations.mappers.ValueMapper;
+import org.apache.storm.streams.processors.BranchProcessor;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.IRichBolt;
+import org.apache.storm.topology.IRichSpout;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseRichBolt;
+import org.apache.storm.topology.base.BaseRichSpout;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.utils.Utils;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Unit tests for {@link StreamBuilder}
+ */
+public class StreamBuilderTest {
+    StreamBuilder streamBuilder;
+
+    @Before
+    public void setUp() throws Exception {
+        streamBuilder = new StreamBuilder();
+        UniqueIdGen.getInstance().reset();
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testSpoutNoDefaultStream() throws Exception {
+        Stream<Tuple> stream = streamBuilder.newStream(newSpout("test"));
+        stream.filter(x -> true);
+        streamBuilder.build();
+    }
+
+    @Test
+    public void testSpoutToBolt() throws Exception {
+        Stream<Tuple> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID));
+        stream.to(newBolt());
+        StormTopology topology = streamBuilder.build();
+        assertEquals(1, topology.get_spouts_size());
+        assertEquals(1, topology.get_bolts_size());
+        String spoutId = topology.get_spouts().keySet().iterator().next();
+        Map<GlobalStreamId, Grouping> expected = new HashMap<>();
+        expected.put(new GlobalStreamId(spoutId, "default"), Grouping.shuffle(new NullStruct()));
+        assertEquals(expected, topology.get_bolts().values().iterator().next().get_common().get_inputs());
+    }
+
+    @Test
+    public void testBranch() throws Exception {
+        Stream<Tuple> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID));
+        Stream<Tuple>[] streams = stream.branch(x -> true);
+        StormTopology topology = streamBuilder.build();
+        assertEquals(1, topology.get_spouts_size());
+        assertEquals(1, topology.get_bolts_size());
+        Map<GlobalStreamId, Grouping> expected = new HashMap<>();
+        String spoutId = topology.get_spouts().keySet().iterator().next();
+        expected.put(new GlobalStreamId(spoutId, "default"), Grouping.shuffle(new NullStruct()));
+        assertEquals(expected, topology.get_bolts().values().iterator().next().get_common().get_inputs());
+        assertEquals(1, streams.length);
+        assertEquals(1, streams[0].node.getOutputStreams().size());
+        String parentStream = streams[0].node.getOutputStreams().iterator().next() + "-branch";
+        assertEquals(1, streams[0].node.getParents(parentStream).size());
+        Node processorNdoe = streams[0].node.getParents(parentStream).iterator().next();
+        assertTrue(processorNdoe instanceof ProcessorNode);
+        assertTrue(((ProcessorNode) processorNdoe).getProcessor() instanceof BranchProcessor);
+        assertTrue(processorNdoe.getParents("default").iterator().next() instanceof SpoutNode);
+    }
+
+    @Test
+    public void testJoin() throws Exception {
+        Stream<Integer> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0));
+        Stream<Integer>[] streams = stream.branch(x -> x % 2 == 0, x-> x % 3 == 0);
+        PairStream<Integer, Integer> s1 = streams[0].mapToPair(x -> Pair.of(x, 1));
+        PairStream<Integer, Integer> s2 = streams[1].mapToPair(x -> Pair.of(x, 1));
+        PairStream<Integer, Pair<Integer, Integer>> sj = s1.join(s2);
+        assertEquals(Collections.singleton(s1.node), sj.node.getParents(s1.stream));
+        assertEquals(Collections.singleton(s2.node), sj.node.getParents(s2.stream));
+    }
+
+    @Test
+    public void testGroupBy() throws Exception {
+        PairStream<String, String> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new PairValueMapper<>(0, 1));
+
+        stream.groupByKey().aggregateByKey(new Count<>());
+
+        StormTopology topology = streamBuilder.build();
+        assertEquals(2, topology.get_bolts_size());
+        Bolt bolt1 = topology.get_bolts().get("bolt1");
+        Bolt bolt2 = topology.get_bolts().get("bolt2");
+        assertEquals(Grouping.shuffle(new NullStruct()), bolt1.get_common().get_inputs().values().iterator().next());
+        assertEquals(Grouping.fields(Collections.singletonList("key")), bolt2.get_common().get_inputs().values().iterator().next());
+    }
+
+    @Test
+    public void testGlobalAggregate() throws Exception {
+        Stream<String> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0));
+
+        stream.aggregate(new Count<>());
+
+        StormTopology topology = streamBuilder.build();
+        assertEquals(2, topology.get_bolts_size());
+        Bolt bolt1 = topology.get_bolts().get("bolt1");
+        Bolt bolt2 = topology.get_bolts().get("bolt2");
+        String spoutId = topology.get_spouts().keySet().iterator().next();
+        Map<GlobalStreamId, Grouping> expected1 = new HashMap<>();
+        expected1.put(new GlobalStreamId(spoutId, "default"), Grouping.shuffle(new NullStruct()));
+        Map<GlobalStreamId, Grouping> expected2 = new HashMap<>();
+        expected2.put(new GlobalStreamId("bolt1", "s1"), Grouping.fields(Collections.emptyList()));
+        assertEquals(expected1, bolt1.get_common().get_inputs());
+        assertEquals(expected2, bolt2.get_common().get_inputs());
+    }
+
+    @Test
+    public void testRepartition() throws Exception {
+        Stream<String> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0));
+        stream.repartition(3).filter(x -> true).repartition(2).filter(x -> true).aggregate(new Count<>());
+        StormTopology topology = streamBuilder.build();
+        assertEquals(1, topology.get_spouts_size());
+        SpoutSpec spout = topology.get_spouts().get("spout1");
+        assertEquals(4, topology.get_bolts_size());
+        Bolt bolt1 = topology.get_bolts().get("bolt1");
+        Bolt bolt2 = topology.get_bolts().get("bolt2");
+        Bolt bolt3 = topology.get_bolts().get("bolt3");
+        Bolt bolt4 = topology.get_bolts().get("bolt4");
+        assertEquals(1, spout.get_common().get_parallelism_hint());
+        assertEquals(1, bolt1.get_common().get_parallelism_hint());
+        assertEquals(3, bolt2.get_common().get_parallelism_hint());
+        assertEquals(2, bolt3.get_common().get_parallelism_hint());
+        assertEquals(2, bolt4.get_common().get_parallelism_hint());
+    }
+
+    @Test
+    public void testBranchAndJoin() throws Exception {
+        TopologyContext mockContext = Mockito.mock(TopologyContext.class);
+        OutputCollector mockCollector = Mockito.mock(OutputCollector.class);
+        Stream<Integer> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0));
+        Stream<Integer>[] streams = stream.branch(x -> x % 2 == 0, x -> x % 2 == 1);
+        PairStream<Integer, Pair<Integer, Integer>> joined = streams[0].mapToPair(x -> Pair.of(x, 1)).join(streams[1].mapToPair(x -> Pair.of(x, 1)));
+        assertTrue(joined.getNode() instanceof ProcessorNode);
+        StormTopology topology = streamBuilder.build();
+        assertEquals(1, topology.get_bolts_size());
+    }
+
+    private static IRichSpout newSpout(final String os) {
+        return new BaseRichSpout() {
+
+            @Override
+            public void declareOutputFields(OutputFieldsDeclarer declarer) {
+                declarer.declareStream(os, new Fields("value"));
+            }
+
+            @Override
+            public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+
+            }
+
+            @Override
+            public void nextTuple() {
+
+            }
+        };
+    }
+
+    private static IRichBolt newBolt() {
+        return new BaseRichBolt() {
+
+            @Override
+            public void declareOutputFields(OutputFieldsDeclarer declarer) {
+
+            }
+
+            @Override
+            public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+
+            }
+
+            @Override
+            public void execute(Tuple input) {
+
+            }
+        };
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/test/jvm/org/apache/storm/streams/WindowedProcessorBoltTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/streams/WindowedProcessorBoltTest.java b/storm-core/test/jvm/org/apache/storm/streams/WindowedProcessorBoltTest.java
new file mode 100644
index 0000000..7428e3f
--- /dev/null
+++ b/storm-core/test/jvm/org/apache/storm/streams/WindowedProcessorBoltTest.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.streams;
+
+import com.google.common.collect.Multimap;
+import org.apache.storm.streams.operations.aggregators.Count;
+import org.apache.storm.streams.processors.AggregateProcessor;
+import org.apache.storm.streams.processors.Processor;
+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.base.BaseWindowedBolt;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.windowing.TupleWindow;
+import org.jgrapht.DirectedGraph;
+import org.jgrapht.graph.DefaultDirectedGraph;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Unit test for {@link WindowedProcessorBolt}
+ */
+public class WindowedProcessorBoltTest {
+    TopologyContext mockTopologyContext;
+    OutputCollector mockOutputCollector;
+    WindowedProcessorBolt bolt;
+    Tuple mockTuple1;
+    Tuple mockTuple2;
+    Tuple mockTuple3;
+    DirectedGraph<Node, Edge> graph;
+    Multimap<String, ProcessorNode> mockStreamToProcessors;
+
+    @Before
+    public void setUp() throws Exception {
+        mockTopologyContext = Mockito.mock(TopologyContext.class);
+        mockOutputCollector = Mockito.mock(OutputCollector.class);
+        mockTuple1 = Mockito.mock(Tuple.class);
+        mockTuple2 = Mockito.mock(Tuple.class);
+        mockTuple3 = Mockito.mock(Tuple.class);
+        setUpMockTuples(mockTuple1, mockTuple2, mockTuple3);
+        mockStreamToProcessors = Mockito.mock(Multimap.class);
+    }
+
+    @Test
+    public void testEmit() throws Exception {
+        Window<?, ?> window = TumblingWindows.of(BaseWindowedBolt.Count.of(2));
+        setUpWindowedProcessorBolt(new AggregateProcessor<>(new Count<>()), window);
+        bolt.execute(getMockTupleWindow(mockTuple1, mockTuple2, mockTuple3));
+        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
+        ArgumentCaptor<String> os = ArgumentCaptor.forClass(String.class);
+        Mockito.verify(mockOutputCollector, Mockito.times(2)).emit(os.capture(), values.capture());
+        assertEquals("outputstream", os.getAllValues().get(0));
+        assertEquals(new Values(3L), values.getAllValues().get(0));
+        assertEquals("outputstream", os.getAllValues().get(1));
+        assertEquals(new Values(WindowNode.PUNCTUATION), values.getAllValues().get(1));
+    }
+
+    private void setUpWindowedProcessorBolt(Processor<?> processor, Window<?, ?> window) {
+        ProcessorNode node = new ProcessorNode(processor, "outputstream", new Fields("value"));
+        node.setWindowed(true);
+        Mockito.when(mockStreamToProcessors.get(Mockito.anyString())).thenReturn(Collections.singletonList(node));
+        Mockito.when(mockStreamToProcessors.keySet()).thenReturn(Collections.singleton("inputstream"));
+        graph = new DefaultDirectedGraph<>(new StreamsEdgeFactory());
+        graph.addVertex(node);
+        bolt = new WindowedProcessorBolt("bolt1", graph, Collections.singletonList(node), window);
+        bolt.setStreamToInitialProcessors(mockStreamToProcessors);
+        bolt.prepare(new HashMap<>(), mockTopologyContext, mockOutputCollector);
+    }
+
+    private void setUpMockTuples(Tuple... tuples) {
+        for (Tuple tuple : tuples) {
+            Mockito.when(tuple.size()).thenReturn(1);
+            Mockito.when(tuple.getValue(0)).thenReturn(100);
+            Mockito.when(tuple.getSourceComponent()).thenReturn("bolt0");
+            Mockito.when(tuple.getSourceStreamId()).thenReturn("inputstream");
+        }
+    }
+
+    private TupleWindow getMockTupleWindow(Tuple... tuples) {
+        TupleWindow tupleWindow = Mockito.mock(TupleWindow.class);
+        Mockito.when(tupleWindow.get()).thenReturn(Arrays.asList(tuples));
+        return tupleWindow;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/test/jvm/org/apache/storm/windowing/WindowManagerTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/windowing/WindowManagerTest.java b/storm-core/test/jvm/org/apache/storm/windowing/WindowManagerTest.java
index 6645566..6c170c6 100644
--- a/storm-core/test/jvm/org/apache/storm/windowing/WindowManagerTest.java
+++ b/storm-core/test/jvm/org/apache/storm/windowing/WindowManagerTest.java
@@ -60,7 +60,7 @@ public class WindowManagerTest {
         }
 
         @Override
-        public void onActivation(List<Integer> events, List<Integer> newEvents, List<Integer> expired) {
+        public void onActivation(List<Integer> events, List<Integer> newEvents, List<Integer> expired, Long timestamp) {
             onActivationEvents = events;
             allOnActivationEvents.add(events);
             onActivationNewEvents = newEvents;


[09/12] storm git commit: [STORM-1961] Added interface stability annotation

Posted by ka...@apache.org.
[STORM-1961] Added interface stability annotation

1. Added interface stability annotation (borrowed from hadoop)
2. Marked relevant stream api classes 'Unstable'


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/a07039fc
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/a07039fc
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/a07039fc

Branch: refs/heads/master
Commit: a07039fc07f6f2aa5d3baeaf5107359218d84b5c
Parents: dc19597
Author: Arun Mahadevan <ar...@apache.org>
Authored: Fri Jan 20 10:33:09 2017 +0530
Committer: Arun Mahadevan <ar...@apache.org>
Committed: Tue Jan 24 17:33:40 2017 +0530

----------------------------------------------------------------------
 .../storm/annotation/InterfaceStability.java    | 54 ++++++++++++++++++++
 .../org/apache/storm/streams/PairStream.java    |  2 +
 .../jvm/org/apache/storm/streams/Stream.java    |  2 +
 .../org/apache/storm/streams/StreamBuilder.java |  2 +
 .../storm/streams/processors/Processor.java     |  3 ++
 .../streams/processors/ProcessorContext.java    |  3 ++
 6 files changed, 66 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/a07039fc/storm-core/src/jvm/org/apache/storm/annotation/InterfaceStability.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/annotation/InterfaceStability.java b/storm-core/src/jvm/org/apache/storm/annotation/InterfaceStability.java
new file mode 100644
index 0000000..d05ae75
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/annotation/InterfaceStability.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.annotation;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+/**
+ * Annotation to inform users of how much to rely on a particular package,
+ * class or method not changing over time.
+ * </ul>
+ */
+@InterfaceStability.Evolving
+public class InterfaceStability {
+    /**
+     * Can evolve while retaining compatibility for minor release boundaries.;
+     * can break compatibility only at major release (ie. at m.0).
+     */
+    @Documented
+    @Retention(RetentionPolicy.RUNTIME)
+    public @interface Stable {};
+
+    /**
+     * Evolving, but can break compatibility at minor release (i.e. m.x)
+     */
+    @Documented
+    @Retention(RetentionPolicy.RUNTIME)
+    public @interface Evolving {};
+
+    /**
+     * No guarantee is provided as to reliability or stability across any
+     * level of release granularity.
+     */
+    @Documented
+    @Retention(RetentionPolicy.RUNTIME)
+    public @interface Unstable {};
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/a07039fc/storm-core/src/jvm/org/apache/storm/streams/PairStream.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/PairStream.java b/storm-core/src/jvm/org/apache/storm/streams/PairStream.java
index 69e6c37..3c08a05 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/PairStream.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/PairStream.java
@@ -18,6 +18,7 @@
 package org.apache.storm.streams;
 
 import org.apache.storm.Config;
+import org.apache.storm.annotation.InterfaceStability;
 import org.apache.storm.streams.operations.BiFunction;
 import org.apache.storm.streams.operations.CombinerAggregator;
 import org.apache.storm.streams.operations.Consumer;
@@ -50,6 +51,7 @@ import java.util.Set;
  * @param <K> the key type
  * @param <V> the value type
  */
+@InterfaceStability.Unstable
 public class PairStream<K, V> extends Stream<Pair<K, V>> {
 
     PairStream(StreamBuilder topology, Node node) {

http://git-wip-us.apache.org/repos/asf/storm/blob/a07039fc/storm-core/src/jvm/org/apache/storm/streams/Stream.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/Stream.java b/storm-core/src/jvm/org/apache/storm/streams/Stream.java
index ef03ae3..087e760 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/Stream.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/Stream.java
@@ -17,6 +17,7 @@
  */
 package org.apache.storm.streams;
 
+import org.apache.storm.annotation.InterfaceStability;
 import org.apache.storm.streams.operations.BiFunction;
 import org.apache.storm.streams.operations.CombinerAggregator;
 import org.apache.storm.streams.operations.Consumer;
@@ -58,6 +59,7 @@ import java.util.Set;
  *
  * @param <T> the type of the value
  */
+@InterfaceStability.Unstable
 public class Stream<T> {
     private static final Logger LOG = LoggerFactory.getLogger(Stream.class);
 

http://git-wip-us.apache.org/repos/asf/storm/blob/a07039fc/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java b/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java
index 0bf02be..8b6c57d 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java
@@ -21,6 +21,7 @@ import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.HashBasedTable;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Table;
+import org.apache.storm.annotation.InterfaceStability;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.streams.operations.IdentityFunction;
 import org.apache.storm.streams.operations.mappers.PairValueMapper;
@@ -58,6 +59,7 @@ import java.util.stream.Collectors;
 /**
  * A builder for constructing a {@link StormTopology} via storm streams api (DSL)
  */
+@InterfaceStability.Unstable
 public class StreamBuilder {
     private static final Logger LOG = LoggerFactory.getLogger(StreamBuilder.class);
     private final DefaultDirectedGraph<Node, Edge> graph;

http://git-wip-us.apache.org/repos/asf/storm/blob/a07039fc/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java
index 5551682..3b4518d 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java
@@ -17,6 +17,8 @@
  */
 package org.apache.storm.streams.processors;
 
+import org.apache.storm.annotation.InterfaceStability;
+
 import java.io.Serializable;
 
 /**
@@ -24,6 +26,7 @@ import java.io.Serializable;
  *
  * @param <T> the type of the input that is processed
  */
+@InterfaceStability.Unstable
 public interface Processor<T> extends Serializable {
     /**
      * Initializes the processor. This is typically invoked from the underlying

http://git-wip-us.apache.org/repos/asf/storm/blob/a07039fc/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java
index 2a10a22..a69a0a9 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java
@@ -17,12 +17,15 @@
  */
 package org.apache.storm.streams.processors;
 
+import org.apache.storm.annotation.InterfaceStability;
+
 import java.io.Serializable;
 import java.util.Set;
 
 /**
  * Context information passed to the {@link Processor}.
  */
+@InterfaceStability.Unstable
 public interface ProcessorContext extends Serializable {
     /**
      * Forwards the input to all downstream processors.


[05/12] storm git commit: [STORM-1961] Fields grouping for state query and refactored StreamBuilder

Posted by ka...@apache.org.
[STORM-1961] Fields grouping for state query and refactored StreamBuilder


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/dc19597c
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/dc19597c
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/dc19597c

Branch: refs/heads/master
Commit: dc19597c290ca0177cf9a06c81f13a60a3afedd2
Parents: 3a10865
Author: Arun Mahadevan <ar...@apache.org>
Authored: Thu Dec 29 15:16:34 2016 +0530
Committer: Arun Mahadevan <ar...@apache.org>
Committed: Fri Jan 13 01:20:44 2017 +0530

----------------------------------------------------------------------
 .../starter/streams/StateQueryExample.java      |   2 +-
 .../src/jvm/org/apache/storm/streams/Node.java  |   2 +-
 .../org/apache/storm/streams/PairStream.java    |   8 +-
 .../jvm/org/apache/storm/streams/Stream.java    |  16 ++-
 .../org/apache/storm/streams/StreamBuilder.java | 113 +++++++++----------
 5 files changed, 63 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/dc19597c/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java
----------------------------------------------------------------------
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
index 2f0a4a3..6d6a4b3 100644
--- 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
@@ -53,7 +53,7 @@ import java.util.Map;
 public class StateQueryExample {
     public static void main(String[] args) throws Exception {
         StreamBuilder builder = new StreamBuilder();
-        StreamState<String, Long> ss = builder.newStream(new TestWordSpout(), new ValueMapper<String>(0))
+        StreamState<String, Long> ss = builder.newStream(new TestWordSpout(), new ValueMapper<String>(0), 2)
                 .mapToPair(w -> Pair.of(w, 1))
                 .updateStateByKey(0L, (count, val) -> count + 1);
 

http://git-wip-us.apache.org/repos/asf/storm/blob/dc19597c/storm-core/src/jvm/org/apache/storm/streams/Node.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/Node.java b/storm-core/src/jvm/org/apache/storm/streams/Node.java
index 3507f50..d21dee9 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/Node.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/Node.java
@@ -91,7 +91,7 @@ abstract class Node implements Serializable {
         this.componentId = componentId;
     }
 
-    Integer getParallelism() {
+    int getParallelism() {
         return parallelism;
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/dc19597c/storm-core/src/jvm/org/apache/storm/streams/PairStream.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/PairStream.java b/storm-core/src/jvm/org/apache/storm/streams/PairStream.java
index 964cdba..69e6c37 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/PairStream.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/PairStream.java
@@ -420,13 +420,7 @@ public class PairStream<K, V> extends Stream<Pair<K, V>> {
     }
 
     private PairStream<K, V> partitionBy(Fields fields) {
-        return partitionBy(fields, node.parallelism);
-    }
-
-    private PairStream<K, V> partitionBy(Fields fields, int parallelism) {
-        return new PairStream<>(
-                streamBuilder,
-                addNode(node, new PartitionNode(stream, node.getOutputFields(), GroupingInfo.fields(fields)), parallelism));
+        return toPairStream(partitionBy(fields, node.parallelism));
     }
 
     private PairStream<K, V> toPairStream(Stream<Pair<K, V>> stream) {

http://git-wip-us.apache.org/repos/asf/storm/blob/dc19597c/storm-core/src/jvm/org/apache/storm/streams/Stream.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/Stream.java b/storm-core/src/jvm/org/apache/storm/streams/Stream.java
index d553390..ef03ae3 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/Stream.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/Stream.java
@@ -370,9 +370,9 @@ public class Stream<T> {
      * @return the result stream
      */
     public <V> PairStream<T, V> stateQuery(StreamState<T, V> streamState) {
-        // need all grouping for state query since the state is per-task
-        Node node = all().addProcessorNode(new StateQueryProcessor<>(streamState), KEY_VALUE);
-        return new PairStream<>(streamBuilder, node);
+        // need field grouping for state query so that the query is routed to the correct task
+        Node newNode = partitionBy(VALUE, node.getParallelism()).addProcessorNode(new StateQueryProcessor<>(streamState), KEY_VALUE);
+        return new PairStream<>(streamBuilder, newNode);
     }
 
     Node getNode() {
@@ -435,12 +435,10 @@ public class Stream<T> {
         return new Stream<>(streamBuilder, partitionNode);
     }
 
-    private Stream<T> all() {
-        if (node.getParallelism() == 1) {
-            return this;
-        }
-        Node partitionNode = addNode(new PartitionNode(stream, node.getOutputFields(), GroupingInfo.all()));
-        return new Stream<>(streamBuilder, partitionNode);
+    protected Stream<T> partitionBy(Fields fields, int parallelism) {
+        return new Stream<>(
+                streamBuilder,
+                addNode(node, new PartitionNode(stream, node.getOutputFields(), GroupingInfo.fields(fields)), parallelism));
     }
 
     private boolean shouldPartition() {

http://git-wip-us.apache.org/repos/asf/storm/blob/dc19597c/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java b/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java
index 7dff25d..0bf02be 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java
@@ -17,10 +17,7 @@
  */
 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;
@@ -28,7 +25,6 @@ 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.Processor;
 import org.apache.storm.streams.processors.StateQueryProcessor;
@@ -48,6 +44,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -56,6 +53,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.PriorityQueue;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 /**
  * A builder for constructing a {@link StormTopology} via storm streams api (DSL)
@@ -261,9 +259,9 @@ public class StreamBuilder {
                 p.put(SpoutNode.class, 0);
                 p.put(UpdateStateByKeyProcessor.class, 1);
                 p.put(ProcessorNode.class, 2);
-                p.put(StateQueryProcessor.class, 3);
-                p.put(PartitionNode.class, 4);
-                p.put(WindowNode.class, 5);
+                p.put(PartitionNode.class, 3);
+                p.put(WindowNode.class, 4);
+                p.put(StateQueryProcessor.class, 5);
                 p.put(SinkNode.class, 6);
             }
             @Override
@@ -384,30 +382,36 @@ public class StreamBuilder {
         return nodes;
     }
 
+    private Collection<List<ProcessorNode>> parallelismGroups(List<ProcessorNode> processorNodes) {
+        return processorNodes.stream().collect(Collectors.groupingBy(Node::getParallelism)).values();
+    }
+
     private void processCurGroup(TopologyBuilder topologyBuilder) {
-        if (curGroup.isEmpty()) {
-            return;
+        if (!curGroup.isEmpty()) {
+            parallelismGroups(curGroup).forEach(g -> doProcessCurGroup(topologyBuilder, g));
+            curGroup.clear();
         }
+    }
 
+    private void doProcessCurGroup(TopologyBuilder topologyBuilder, List<ProcessorNode> group) {
         String boltId = UniqueIdGen.getInstance().getUniqueBoltId();
-        for (ProcessorNode processorNode : curGroup) {
+        for (ProcessorNode processorNode : group) {
             processorNode.setComponentId(boltId);
             processorNode.setWindowedParentStreams(getWindowedParentStreams(processorNode));
         }
-        final Set<ProcessorNode> initialProcessors = initialProcessors(curGroup);
+        final Set<ProcessorNode> initialProcessors = initialProcessors(group);
         Set<Window<?, ?>> windowParams = getWindowParams(initialProcessors);
         if (windowParams.isEmpty()) {
-            if (hasStatefulProcessor(curGroup)) {
-                addStatefulBolt(topologyBuilder, boltId, initialProcessors);
+            if (hasStatefulProcessor(group)) {
+                addStatefulBolt(topologyBuilder, boltId, initialProcessors, group);
             } else {
-                addBolt(topologyBuilder, boltId, initialProcessors);
+                addBolt(topologyBuilder, boltId, initialProcessors, group);
             }
         } else if (windowParams.size() == 1) {
-            addWindowedBolt(topologyBuilder, boltId, initialProcessors, windowParams.iterator().next());
+            addWindowedBolt(topologyBuilder, boltId, initialProcessors, windowParams.iterator().next(), group);
         } else {
-            throw new IllegalStateException("More than one window config for current group " + curGroup);
+            throw new IllegalStateException("More than one window config for current group " + group);
         }
-        curGroup.clear();
     }
 
     private boolean hasStatefulProcessor(List<ProcessorNode> processorNodes) {
@@ -419,16 +423,11 @@ public class StreamBuilder {
         return false;
     }
 
-    private int getParallelism() {
-        Set<Integer> parallelisms = new HashSet<>(Collections2.transform(curGroup, new Function<ProcessorNode, Integer>() {
-            @Override
-            public Integer apply(ProcessorNode input) {
-                return input.getParallelism();
-            }
-        }));
+    private int getParallelism(List<ProcessorNode> group) {
+        Set<Integer> parallelisms = group.stream().map(Node::getParallelism).collect(Collectors.toSet());
 
         if (parallelisms.size() > 1) {
-            throw new IllegalStateException("Current group does not have same parallelism " + curGroup);
+            throw new IllegalStateException("Current group does not have same parallelism " + group);
         }
 
         return parallelisms.isEmpty() ? 1 : parallelisms.iterator().next();
@@ -446,16 +445,7 @@ public class StreamBuilder {
             }
         }
 
-        Set<Window<?, ?>> windowParams = new HashSet<>();
-        if (!windowNodes.isEmpty()) {
-            windowParams.addAll(new HashSet<>(Collections2.transform(windowNodes, new Function<WindowNode, Window<?, ?>>() {
-                @Override
-                public Window<?, ?> apply(WindowNode input) {
-                    return input.getWindowParams();
-                }
-            })));
-        }
-        return windowParams;
+        return windowNodes.stream().map(WindowNode::getWindowParams).collect(Collectors.toSet());
     }
 
     private void addSpout(TopologyBuilder topologyBuilder, SpoutNode spout) {
@@ -481,39 +471,41 @@ public class StreamBuilder {
 
     private StreamBolt addBolt(TopologyBuilder topologyBuilder,
                                String boltId,
-                               Set<ProcessorNode> initialProcessors) {
-        ProcessorBolt bolt = new ProcessorBolt(boltId, graph, curGroup);
-        BoltDeclarer boltDeclarer = topologyBuilder.setBolt(boltId, bolt, getParallelism());
-        bolt.setStreamToInitialProcessors(wireBolt(curGroup, boltDeclarer, initialProcessors));
+                               Set<ProcessorNode> initialProcessors,
+                               List<ProcessorNode> group) {
+        ProcessorBolt bolt = new ProcessorBolt(boltId, graph, group);
+        BoltDeclarer boltDeclarer = topologyBuilder.setBolt(boltId, bolt, getParallelism(group));
+        bolt.setStreamToInitialProcessors(wireBolt(group, boltDeclarer, initialProcessors));
         streamBolts.put(bolt, boltDeclarer);
         return bolt;
     }
 
     private StreamBolt addStatefulBolt(TopologyBuilder topologyBuilder,
                                        String boltId,
-                                       Set<ProcessorNode> initialProcessors) {
-        StateQueryProcessor<?, ?> stateQueryProcessor = getStateQueryProcessor();
+                                       Set<ProcessorNode> initialProcessors,
+                                       List<ProcessorNode> group) {
+        StateQueryProcessor<?, ?> stateQueryProcessor = getStateQueryProcessor(group);
         StatefulProcessorBolt<?, ?> bolt;
         if (stateQueryProcessor == null) {
-            bolt = new StatefulProcessorBolt<>(boltId, graph, curGroup);
-            BoltDeclarer boltDeclarer = topologyBuilder.setBolt(boltId, bolt, getParallelism());
-            bolt.setStreamToInitialProcessors(wireBolt(curGroup, boltDeclarer, initialProcessors));
+            bolt = new StatefulProcessorBolt<>(boltId, graph, group);
+            BoltDeclarer boltDeclarer = topologyBuilder.setBolt(boltId, bolt, getParallelism(group));
+            bolt.setStreamToInitialProcessors(wireBolt(group, 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) {
+            for (ProcessorNode node : group) {
                 node.setComponentId(bolt.getId());
             }
-            bolt.addNodes(curGroup);
+            bolt.addNodes(group);
             bolt.addStreamToInitialProcessors(wireBolt(bolt.getNodes(), streamBolts.get(bolt), initialProcessors));
         }
         return bolt;
     }
 
-    private StateQueryProcessor<?, ?> getStateQueryProcessor() {
-        for (ProcessorNode node : curGroup) {
+    private StateQueryProcessor<?, ?> getStateQueryProcessor(List<ProcessorNode> group) {
+        for (ProcessorNode node : group) {
             if (node.getProcessor() instanceof StateQueryProcessor) {
                 return (StateQueryProcessor<?, ?>) node.getProcessor();
             }
@@ -524,10 +516,11 @@ public class StreamBuilder {
     private StreamBolt addWindowedBolt(TopologyBuilder topologyBuilder,
                                        String boltId,
                                        Set<ProcessorNode> initialProcessors,
-                                       Window<?, ?> windowParam) {
-        WindowedProcessorBolt bolt = new WindowedProcessorBolt(boltId, graph, curGroup, windowParam);
-        BoltDeclarer boltDeclarer = topologyBuilder.setBolt(boltId, bolt, getParallelism());
-        bolt.setStreamToInitialProcessors(wireBolt(curGroup, boltDeclarer, initialProcessors));
+                                       Window<?, ?> windowParam,
+                                       List<ProcessorNode> group) {
+        WindowedProcessorBolt bolt = new WindowedProcessorBolt(boltId, graph, group, windowParam);
+        BoltDeclarer boltDeclarer = topologyBuilder.setBolt(boltId, bolt, getParallelism(group));
+        bolt.setStreamToInitialProcessors(wireBolt(group, boltDeclarer, initialProcessors));
         streamBolts.put(bolt, boltDeclarer);
         return bolt;
     }
@@ -554,17 +547,17 @@ public class StreamBuilder {
         return res;
     }
 
-    private Multimap<String, ProcessorNode> wireBolt(List<ProcessorNode> curGroup,
+    private Multimap<String, ProcessorNode> wireBolt(List<ProcessorNode> group,
                                                      BoltDeclarer boltDeclarer,
                                                      Set<ProcessorNode> initialProcessors) {
-        LOG.debug("Wiring bolt with boltDeclarer {}, curGroup {}, initialProcessors {}, nodeGroupingInfo {}",
-                boltDeclarer, curGroup, initialProcessors, nodeGroupingInfo);
+        LOG.debug("Wiring bolt with boltDeclarer {}, group {}, initialProcessors {}, nodeGroupingInfo {}",
+                boltDeclarer, group, initialProcessors, nodeGroupingInfo);
         Multimap<String, ProcessorNode> streamToInitialProcessor = ArrayListMultimap.create();
-        Set<ProcessorNode> curSet = new HashSet<>(curGroup);
+        Set<ProcessorNode> curSet = new HashSet<>(group);
         for (ProcessorNode curNode : initialProcessors) {
             for (Node parent : parentNodes(curNode)) {
                 if (curSet.contains(parent)) {
-                    LOG.debug("Parent {} of curNode {} is in curGroup {}", parent, curNode, curGroup);
+                    LOG.debug("Parent {} of curNode {} is in group {}", parent, curNode, group);
                 } else {
                     for (String stream : curNode.getParentStreams(parent)) {
                         declareGrouping(boltDeclarer, parent, stream, nodeGroupingInfo.get(parent, stream));
@@ -592,10 +585,10 @@ public class StreamBuilder {
         }
     }
 
-    private Set<ProcessorNode> initialProcessors(List<ProcessorNode> curGroup) {
+    private Set<ProcessorNode> initialProcessors(List<ProcessorNode> group) {
         Set<ProcessorNode> nodes = new HashSet<>();
-        Set<ProcessorNode> curSet = new HashSet<>(curGroup);
-        for (ProcessorNode node : curGroup) {
+        Set<ProcessorNode> curSet = new HashSet<>(group);
+        for (ProcessorNode node : group) {
             for (Node parent : parentNodes(node)) {
                 if (!(parent instanceof ProcessorNode) || !curSet.contains(parent)) {
                     nodes.add(node);


[08/12] storm git commit: [STORM-1961] A few fixes and refactoring

Posted by ka...@apache.org.
[STORM-1961] A few fixes and refactoring

1. Added typed tuples
2. Change groupByKey semantics and refactor examples
3. Handle punctuations correctly
4. Added countByKey and count
5. Added left, right and full outer joins
6. Per partition combine for aggregate/reduce


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/3a10865c
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/3a10865c
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/3a10865c

Branch: refs/heads/master
Commit: 3a10865c628fa0606456826e28ce8838baf60134
Parents: e251573
Author: Arun Mahadevan <ar...@apache.org>
Authored: Tue Oct 25 01:13:53 2016 +0530
Committer: Arun Mahadevan <ar...@apache.org>
Committed: Fri Jan 13 01:20:44 2017 +0530

----------------------------------------------------------------------
 .../storm/starter/streams/AggregateExample.java |  96 ++++++
 .../starter/streams/StateQueryExample.java      |   4 +-
 .../starter/streams/StatefulWordCount.java      |  16 +-
 .../starter/streams/TypedTupleExample.java      |  64 ++++
 .../starter/streams/WindowedWordCount.java      |  18 +-
 .../storm/starter/streams/WordCountToBolt.java  |   7 +-
 .../org/apache/storm/streams/GroupingInfo.java  |  16 +
 .../src/jvm/org/apache/storm/streams/Node.java  |  53 ++-
 .../src/jvm/org/apache/storm/streams/Pair.java  |  19 +
 .../org/apache/storm/streams/PairStream.java    | 345 +++++++++++++++++--
 .../org/apache/storm/streams/PartitionNode.java |   7 +-
 .../storm/streams/ProcessorBoltDelegate.java    | 111 ++++--
 .../org/apache/storm/streams/ProcessorNode.java |  24 +-
 .../jvm/org/apache/storm/streams/Stream.java    | 177 ++++++++--
 .../org/apache/storm/streams/StreamBuilder.java | 130 ++++---
 .../org/apache/storm/streams/StreamUtil.java    |  25 +-
 .../jvm/org/apache/storm/streams/Tuple3.java    |  49 ---
 .../org/apache/storm/streams/WindowNode.java    |   1 +
 .../storm/streams/operations/Aggregator.java    |  42 ---
 .../storm/streams/operations/BiFunction.java    |  37 ++
 .../streams/operations/CombinerAggregator.java  |  97 ++++++
 .../storm/streams/operations/Reducer.java       |   2 +-
 .../storm/streams/operations/StateUpdater.java  |  67 ++++
 .../streams/operations/aggregators/Count.java   |  18 +-
 .../streams/operations/aggregators/Sum.java     |  16 +-
 .../operations/mappers/TupleValueMappers.java   | 174 ++++++++++
 .../processors/AggregateByKeyProcessor.java     |  47 ++-
 .../streams/processors/AggregateProcessor.java  |  44 ++-
 .../streams/processors/BranchProcessor.java     |   6 +-
 .../processors/EmittingProcessorContext.java    |  35 +-
 .../storm/streams/processors/JoinProcessor.java |  54 ++-
 .../MergeAggregateByKeyProcessor.java           |  54 +++
 .../processors/MergeAggregateProcessor.java     |  47 +++
 .../streams/processors/ReduceProcessor.java     |   6 +-
 .../processors/UpdateStateByKeyProcessor.java   |  12 +-
 .../org/apache/storm/streams/tuple/Tuple10.java | 112 ++++++
 .../org/apache/storm/streams/tuple/Tuple3.java  |  70 ++++
 .../org/apache/storm/streams/tuple/Tuple4.java  |  76 ++++
 .../org/apache/storm/streams/tuple/Tuple5.java  |  82 +++++
 .../org/apache/storm/streams/tuple/Tuple6.java  |  89 +++++
 .../org/apache/storm/streams/tuple/Tuple7.java  |  94 +++++
 .../org/apache/storm/streams/tuple/Tuple8.java  | 100 ++++++
 .../org/apache/storm/streams/tuple/Tuple9.java  | 106 ++++++
 .../apache/storm/streams/ProcessorBoltTest.java |  13 +-
 .../streams/StatefulProcessorBoltTest.java      |  15 +-
 .../apache/storm/streams/StreamBuilderTest.java |  57 ++-
 .../streams/WindowedProcessorBoltTest.java      |   2 +-
 .../streams/processors/JoinProcessorTest.java   | 108 ++++++
 48 files changed, 2471 insertions(+), 373 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java
new file mode 100644
index 0000000..91dfadb
--- /dev/null
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java
@@ -0,0 +1,96 @@
+/**
+ * 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.Pair;
+import org.apache.storm.streams.PairStream;
+import org.apache.storm.streams.Stream;
+import org.apache.storm.streams.StreamBuilder;
+import org.apache.storm.streams.operations.CombinerAggregator;
+import org.apache.storm.streams.operations.mappers.TupleValueMapper;
+import org.apache.storm.streams.operations.mappers.TupleValueMappers;
+import org.apache.storm.streams.operations.mappers.ValueMapper;
+import org.apache.storm.streams.tuple.Tuple3;
+import org.apache.storm.streams.windowing.TumblingWindows;
+import org.apache.storm.topology.base.BaseWindowedBolt;
+import org.apache.storm.trident.windowing.config.TumblingDurationWindow;
+import org.apache.storm.utils.Utils;
+
+import static org.apache.storm.topology.base.BaseWindowedBolt.Count;
+
+/**
+ * An example that illustrates the global aggregate
+ */
+public class AggregateExample {
+    @SuppressWarnings("unchecked")
+    public static void main(String[] args) throws Exception {
+        StreamBuilder builder = new StreamBuilder();
+        /**
+         * Computes average of the stream of numbers emitted by the spout. Internally the per-partition
+         * sum and counts are accumulated and emitted to a downstream task where the partially accumulated
+         * results are merged and the final result is emitted.
+         */
+        builder.newStream(new RandomIntegerSpout(), new ValueMapper<Integer>(0), 2)
+                .window(TumblingWindows.of(BaseWindowedBolt.Duration.seconds(5)))
+                .filter(x -> x > 0 && x < 500)
+                .aggregate(new Avg())
+                .print();
+
+        Config config = new Config();
+        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 Avg implements CombinerAggregator<Integer, Pair<Integer, Integer>, Double> {
+        @Override
+        public Pair<Integer, Integer> init() {
+            return Pair.of(0, 0);
+        }
+
+        @Override
+        public Pair<Integer, Integer> apply(Pair<Integer, Integer> sumAndCount, Integer value) {
+            return Pair.of(sumAndCount.getFirst() + value, sumAndCount.getSecond() + 1);
+        }
+
+        @Override
+        public Pair<Integer, Integer> merge(Pair<Integer, Integer> sumAndCount1, Pair<Integer, Integer> sumAndCount2) {
+            System.out.println("Merge " + sumAndCount1 + " and " + sumAndCount2);
+            return Pair.of(
+                    sumAndCount1.getFirst() + sumAndCount2.getFirst(),
+                    sumAndCount1.getSecond() + sumAndCount2.getSecond()
+            );
+        }
+
+        @Override
+        public Double result(Pair<Integer, Integer> sumAndCount) {
+            return (double) sumAndCount.getFirst()/sumAndCount.getSecond();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java
----------------------------------------------------------------------
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
index e76dd3c..2f0a4a3 100644
--- 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
@@ -25,7 +25,6 @@ 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;
@@ -56,8 +55,7 @@ public class StateQueryExample {
         StreamBuilder builder = new StreamBuilder();
         StreamState<String, Long> ss = builder.newStream(new TestWordSpout(), new ValueMapper<String>(0))
                 .mapToPair(w -> Pair.of(w, 1))
-                .groupByKey()
-                .updateStateByKey(new Count<>());
+                .updateStateByKey(0L, (count, val) -> count + 1);
 
         /*
          * A stream of words emitted by the QuerySpout is used as

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java
----------------------------------------------------------------------
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
index f6ae6b0..ce7470d 100644
--- 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
@@ -23,14 +23,15 @@ 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.StateUpdater;
 import org.apache.storm.streams.operations.mappers.ValueMapper;
+import org.apache.storm.streams.windowing.TumblingWindows;
 import org.apache.storm.testing.TestWordSpout;
+import org.apache.storm.topology.base.BaseWindowedBolt;
 import org.apache.storm.utils.Utils;
 
 /**
- * A stateful word count that uses {@link PairStream#updateStateByKey(Aggregator)} to
+ * A stateful word count that uses {@link PairStream#updateStateByKey(StateUpdater)} to
  * save the counts in a key value state. This example uses Redis state store.
  * <p>
  * You should start a local redis instance before running the 'storm jar' command. By default
@@ -48,19 +49,20 @@ 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<String>(0))
+        builder.newStream(new TestWordSpout(), new ValueMapper<String>(0), 2)
+                .window(TumblingWindows.of(BaseWindowedBolt.Duration.seconds(2)))
                 /*
                  * create a stream of (word, 1) pairs
                  */
                 .mapToPair(w -> Pair.of(w, 1))
                 /*
-                 * group by the word
+                 * compute the word counts in the last two second window
                  */
-                .groupByKey()
+                .countByKey()
                 /*
                  * update the word counts in the state
                  */
-                .updateStateByKey(new Count<>())
+                .updateStateByKey(0L, (x, y) -> x + y)
                  /*
                   * convert the state back to a stream and print the results
                   */

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java
new file mode 100644
index 0000000..193ad661
--- /dev/null
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java
@@ -0,0 +1,64 @@
+/**
+ * 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.Pair;
+import org.apache.storm.streams.PairStream;
+import org.apache.storm.streams.Stream;
+import org.apache.storm.streams.StreamBuilder;
+import org.apache.storm.streams.operations.mappers.TupleValueMappers;
+import org.apache.storm.streams.tuple.Tuple3;
+import org.apache.storm.streams.windowing.TumblingWindows;
+import org.apache.storm.utils.Utils;
+
+import static org.apache.storm.topology.base.BaseWindowedBolt.Count;
+
+/**
+ * An example that illustrates the usage of typed tuples (TupleN<..>) and {@link TupleValueMappers}.
+ */
+public class TypedTupleExample {
+    @SuppressWarnings("unchecked")
+    public static void main(String[] args) throws Exception {
+        StreamBuilder builder = new StreamBuilder();
+        /**
+         * The spout emits sequences of (Integer, Long, Long). TupleValueMapper can be used to extract fields
+         * from the values and produce a stream of typed tuple (Tuple3<Integer, Long, Long> in this case.
+         */
+        Stream<Tuple3<Integer, Long, Long>> stream = builder.newStream(new RandomIntegerSpout(), TupleValueMappers.of(0, 1, 2));
+
+        PairStream<Long, Integer> pairs = stream.mapToPair(t -> Pair.of(t._2 / 10000, t._1));
+
+        pairs.window(TumblingWindows.of(Count.of(10))).groupByKey().print();
+
+        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();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java
----------------------------------------------------------------------
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
index c6e2f4a..0765a74 100644
--- 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
@@ -23,7 +23,6 @@ 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;
@@ -39,12 +38,11 @@ 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<String>(0))
+        builder.newStream(new RandomSentenceSpout(), new ValueMapper<String>(0), 2)
                 /*
-                 * Increase the parallelism of this stream. Further operations
-                 * on this stream will execute at this level of parallelism.
+                 * a two seconds tumbling window
                  */
-                .repartition(2)
+                .window(TumblingWindows.of(Duration.seconds(2)))
                 /*
                  * split the sentences to words
                  */
@@ -54,17 +52,9 @@ public class WindowedWordCount {
                  */
                 .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<>())
+                .countByKey()
                 /*
                  * emit the count for the words that occurred
                  * at-least five times in the last two seconds

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java
----------------------------------------------------------------------
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
index a711696..dd7923a 100644
--- 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
@@ -26,7 +26,6 @@ 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;
@@ -56,13 +55,9 @@ public class WordCountToBolt {
                  */
                 .mapToPair(w -> Pair.of(w, 1))
                 /*
-                 * group by key (word)
-                 */
-                .groupByKey()
-                /*
                  * aggregate the count
                  */
-                .aggregateByKey(new Count<>())
+                .countByKey()
                 /*
                  * The result of aggregation is forwarded to
                  * the RedisStoreBolt. The forwarded tuple is a

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/GroupingInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/GroupingInfo.java b/storm-core/src/jvm/org/apache/storm/streams/GroupingInfo.java
index 81def4b..9e8c893 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/GroupingInfo.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/GroupingInfo.java
@@ -76,6 +76,22 @@ abstract class GroupingInfo implements Serializable {
     }
 
     @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        GroupingInfo that = (GroupingInfo) o;
+
+        return fields != null ? fields.equals(that.fields) : that.fields == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        return fields != null ? fields.hashCode() : 0;
+    }
+
+    @Override
     public String toString() {
         return "GroupingInfo{" +
                 "fields=" + fields +

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/Node.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/Node.java b/storm-core/src/jvm/org/apache/storm/streams/Node.java
index f9de390..3507f50 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/Node.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/Node.java
@@ -38,30 +38,47 @@ import java.util.Set;
 abstract class Node implements Serializable {
     private final Set<String> outputStreams;
     protected final Fields outputFields;
+    protected GroupingInfo groupingInfo;
     protected String componentId;
     protected int parallelism;
     // the parent streams that this node subscribes to
     private final Multimap<Node, String> parentStreams = ArrayListMultimap.create();
+    private boolean windowed;
+    private boolean emitsPair;
 
-    Node(Set<String> outputStreams, Fields outputFields, String componentId, int parallelism) {
+    Node(Set<String> outputStreams, Fields outputFields, String componentId, int parallelism,
+         GroupingInfo groupingInfo) {
         this.outputStreams = new HashSet<>(outputStreams);
         this.outputFields = outputFields;
         this.componentId = componentId;
         this.parallelism = parallelism;
+        this.groupingInfo = groupingInfo;
     }
 
-    Node(String outputStream, Fields outputFields, String componentId, int parallelism) {
-        this(Collections.singleton(outputStream), outputFields, componentId, parallelism);
+    Node(String outputStream, Fields outputFields, String componentId, int parallelism, GroupingInfo groupingInfo) {
+        this(Collections.singleton(outputStream), outputFields, componentId, parallelism, groupingInfo);
     }
 
-    Node(String outputStream, Fields outputFields, String componentId) {
-        this(outputStream, outputFields, componentId, 1);
+    Node(String outputStream, Fields outputFields, String componentId, GroupingInfo groupingInfo) {
+        this(outputStream, outputFields, componentId, 1, groupingInfo);
     }
 
     Node(String outputStream, Fields outputFields) {
         this(outputStream, outputFields, null);
     }
 
+    Node(String outputStream, Fields outputFields, GroupingInfo groupingInfo) {
+        this(outputStream, outputFields, null, groupingInfo);
+    }
+
+    GroupingInfo getGroupingInfo() {
+        return groupingInfo;
+    }
+
+    void setGroupingInfo(GroupingInfo groupingInfo) {
+        this.groupingInfo = groupingInfo;
+    }
+
     public Fields getOutputFields() {
         return outputFields;
     }
@@ -94,6 +111,14 @@ abstract class Node implements Serializable {
         return Collections.unmodifiableSet(outputStreams);
     }
 
+    public boolean isWindowed() {
+        return windowed;
+    }
+
+    public void setWindowed(boolean windowed) {
+        this.windowed = windowed;
+    }
+
     Collection<String> getParentStreams(Node parent) {
         return parentStreams.get(parent);
     }
@@ -103,6 +128,10 @@ abstract class Node implements Serializable {
         return new HashSet<>(rev.get(stream));
     }
 
+    Set<Node> getParents() {
+        return parentStreams.keySet();
+    }
+
     void addOutputStream(String streamId) {
         outputStreams.add(streamId);
     }
@@ -117,13 +146,25 @@ abstract class Node implements Serializable {
         return new Fields();
     }
 
+    public boolean emitsPair() {
+        return emitsPair;
+    }
+
+    public void setEmitsPair(boolean emitsPair) {
+        this.emitsPair = emitsPair;
+    }
+
     @Override
     public String toString() {
         return "Node{" +
-                "outputStreams='" + outputStreams + '\'' +
+                "outputStreams=" + outputStreams +
                 ", outputFields=" + outputFields +
+                ", groupingInfo=" + groupingInfo +
                 ", componentId='" + componentId + '\'' +
                 ", parallelism=" + parallelism +
+                ", parentStreams=" + parentStreams +
+                ", windowed=" + windowed +
+                ", emitsPair=" + emitsPair +
                 '}';
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/Pair.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/Pair.java b/storm-core/src/jvm/org/apache/storm/streams/Pair.java
index 0044359..e5eb792 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/Pair.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/Pair.java
@@ -72,6 +72,25 @@ public final class Pair<T1, T2> implements Serializable {
     }
 
     @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        Pair<?, ?> pair = (Pair<?, ?>) o;
+
+        if (first != null ? !first.equals(pair.first) : pair.first != null) return false;
+        return second != null ? second.equals(pair.second) : pair.second == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = first != null ? first.hashCode() : 0;
+        result = 31 * result + (second != null ? second.hashCode() : 0);
+        return result;
+    }
+
+    @Override
     public String toString() {
         return "(" + first + ", " + second + ')';
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/PairStream.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/PairStream.java b/storm-core/src/jvm/org/apache/storm/streams/PairStream.java
index 2d18b30..964cdba 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/PairStream.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/PairStream.java
@@ -18,26 +18,31 @@
 package org.apache.storm.streams;
 
 import org.apache.storm.Config;
-import org.apache.storm.streams.operations.Aggregator;
+import org.apache.storm.streams.operations.BiFunction;
+import org.apache.storm.streams.operations.CombinerAggregator;
 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.StateUpdater;
 import org.apache.storm.streams.operations.ValueJoiner;
+import org.apache.storm.streams.operations.aggregators.Count;
 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.MergeAggregateByKeyProcessor;
 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;
+import java.util.Optional;
+import java.util.Set;
 
 /**
  * Represents a stream of key-value pairs.
@@ -49,6 +54,7 @@ public class PairStream<K, V> extends Stream<Pair<K, V>> {
 
     PairStream(StreamBuilder topology, Node node) {
         super(topology, node);
+        node.setEmitsPair(true);
     }
 
     /**
@@ -60,7 +66,8 @@ public class PairStream<K, V> extends Stream<Pair<K, V>> {
      * @return the new stream
      */
     public <R> PairStream<K, R> mapValues(Function<? super V, ? extends R> function) {
-        return new PairStream<>(streamBuilder, addProcessorNode(new MapValuesProcessor<>(function), KEY_VALUE));
+        return new PairStream<>(streamBuilder,
+                addProcessorNode(new MapValuesProcessor<>(function), KEY_VALUE, true));
     }
 
     /**
@@ -71,19 +78,45 @@ public class PairStream<K, V> extends Stream<Pair<K, V>> {
      * @param <R>      the result type
      * @return the new stream
      */
-    public <R> PairStream<K, R> flatMapValues(FlatMapFunction<V, R> function) {
-        return new PairStream<>(streamBuilder, addProcessorNode(new FlatMapValuesProcessor<>(function), KEY_VALUE));
+    public <R> PairStream<K, R> flatMapValues(FlatMapFunction<? super V, ? extends R> function) {
+        return new PairStream<>(streamBuilder,
+                addProcessorNode(new FlatMapValuesProcessor<>(function), KEY_VALUE, true));
     }
 
     /**
-     * Aggregates the values for each key of this stream using the given {@link Aggregator}.
+     * Aggregates the values for each key of this stream using the given initial value, accumulator and combiner.
      *
-     * @param aggregator the aggregator
+     * @param initialValue the initial value of the result
+     * @param accumulator  the accumulator
+     * @param combiner     the combiner
+     * @param <R>          the result type
+     * @return the new stream
+     */
+    public <R> PairStream<K, R> aggregateByKey(R initialValue,
+                                               BiFunction<? super R, ? super V, ? extends R> accumulator,
+                                               BiFunction<? super R, ? super R, ? extends R> combiner) {
+        return combineByKey(CombinerAggregator.of(initialValue, accumulator, combiner));
+    }
+
+    /**
+     * Aggregates the values for each key of this stream using the given {@link CombinerAggregator}.
+     *
+     * @param aggregator the combiner aggregator
+     * @param <A>        the accumulator type
      * @param <R>        the result type
      * @return the new stream
      */
-    public <R> PairStream<K, R> aggregateByKey(Aggregator<? super V, ? extends R> aggregator) {
-        return new PairStream<>(streamBuilder, addProcessorNode(new AggregateByKeyProcessor<>(aggregator), KEY_VALUE));
+    public <A, R> PairStream<K, R> aggregateByKey(CombinerAggregator<? super V, A, ? extends R> aggregator) {
+        return combineByKey(aggregator);
+    }
+
+    /**
+     * Counts the values for each key of this stream.
+     *
+     * @return the new stream
+     */
+    public PairStream<K, Long> countByKey() {
+        return aggregateByKey(new Count<>());
     }
 
     /**
@@ -93,7 +126,7 @@ public class PairStream<K, V> extends Stream<Pair<K, V>> {
      * @return the new stream
      */
     public PairStream<K, V> reduceByKey(Reducer<V> reducer) {
-        return new PairStream<>(streamBuilder, addProcessorNode(new ReduceByKeyProcessor<>(reducer), KEY_VALUE));
+        return combineByKey(reducer);
     }
 
     /**
@@ -101,8 +134,8 @@ public class PairStream<K, V> extends Stream<Pair<K, V>> {
      *
      * @return the new stream
      */
-    public PairStream<K, V> groupByKey() {
-        return partitionBy(KEY);
+    public PairStream<K, Iterable<V>> groupByKey() {
+        return partitionByKey().aggregatePartition(new MergeValues<>());
     }
 
     /**
@@ -114,7 +147,7 @@ public class PairStream<K, V> extends Stream<Pair<K, V>> {
      * @return the new stream
      */
     public PairStream<K, Iterable<V>> groupByKeyAndWindow(Window<?, ?> window) {
-        return groupByKey().window(window).aggregateByKey(new MergeValues<>());
+        return partitionByKey().window(window).aggregatePartition(new MergeValues<>());
     }
 
     /**
@@ -126,7 +159,7 @@ public class PairStream<K, V> extends Stream<Pair<K, V>> {
      * @return the new stream
      */
     public PairStream<K, V> reduceByKeyAndWindow(Reducer<V> reducer, Window<?, ?> window) {
-        return groupByKey().window(window).reduceByKey(reducer);
+        return partitionByKey().window(window).reduceByKey(reducer);
     }
 
     /**
@@ -138,9 +171,16 @@ public class PairStream<K, V> extends Stream<Pair<K, V>> {
     }
 
     /**
+     * {@inheritDoc}
+     */
+    public PairStream<K, V> filter(Predicate<? super Pair<K, V>> predicate) {
+        return toPairStream(super.filter(predicate));
+    }
+
+    /**
      * Join the values of this stream with the values having the same key from the other stream.
      * <p>
-     * Note: The parallelism and windowing parameters (if windowed) of this stream is carried forward to the joined stream.
+     * Note: The parallelism of this stream is carried forward to the joined stream.
      * </p>
      *
      * @param otherStream the other stream
@@ -152,9 +192,51 @@ public class PairStream<K, V> extends Stream<Pair<K, V>> {
     }
 
     /**
+     * Does a left outer join of the values of this stream with the values having the same key from the other stream.
+     * <p>
+     * Note: The parallelism of this stream is carried forward to the joined stream.
+     * </p>
+     *
+     * @param otherStream the other stream
+     * @param <V1>        the type of the values in the other stream
+     * @return the new stream
+     */
+    public <V1> PairStream<K, Pair<V, V1>> leftOuterJoin(PairStream<K, V1> otherStream) {
+        return leftOuterJoin(otherStream, new PairValueJoiner<>());
+    }
+
+    /**
+     * Does a right outer join of the values of this stream with the values having the same key from the other stream.
+     * <p>
+     * Note: The parallelism of this stream is carried forward to the joined stream.
+     * </p>
+     *
+     * @param otherStream the other stream
+     * @param <V1>        the type of the values in the other stream
+     * @return the new stream
+     */
+    public <V1> PairStream<K, Pair<V, V1>> rightOuterJoin(PairStream<K, V1> otherStream) {
+        return rightOuterJoin(otherStream, new PairValueJoiner<>());
+    }
+
+    /**
+     * Does a full outer join of the values of this stream with the values having the same key from the other stream.
+     * <p>
+     * Note: The parallelism of this stream is carried forward to the joined stream.
+     * </p>
+     *
+     * @param otherStream the other stream
+     * @param <V1>        the type of the values in the other stream
+     * @return the new stream
+     */
+    public <V1> PairStream<K, Pair<V, V1>> fullOuterJoin(PairStream<K, V1> otherStream) {
+        return fullOuterJoin(otherStream, new PairValueJoiner<>());
+    }
+
+    /**
      * Join the values of this stream with the values having the same key from the other stream.
      * <p>
-     * Note: The parallelism and windowing parameters (if windowed) of this stream is carried forward to the joined stream.
+     * Note: The parallelism of this stream is carried forward to the joined stream.
      * </p>
      *
      * @param otherStream the other stream
@@ -165,11 +247,78 @@ public class PairStream<K, V> extends Stream<Pair<K, V>> {
      */
     public <R, V1> PairStream<K, R> join(PairStream<K, V1> 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);
+        return partitionByKey()
+                .joinPartition(
+                        otherStream.partitionByKey(),
+                        valueJoiner,
+                        JoinProcessor.JoinType.INNER,
+                        JoinProcessor.JoinType.INNER);
+    }
+
+    /**
+     * Does a left outer join of the values of this stream with the values having the same key from the other stream.
+     * <p>
+     * Note: The parallelism of this stream is carried forward to the joined stream.
+     * </p>
+     *
+     * @param otherStream the other stream
+     * @param valueJoiner the {@link ValueJoiner}
+     * @param <R>         the type of the values resulting from the join
+     * @param <V1>        the type of the values in the other stream
+     * @return the new stream
+     */
+    public <R, V1> PairStream<K, R> leftOuterJoin(PairStream<K, V1> otherStream,
+                                         ValueJoiner<? super V, ? super V1, ? extends R> valueJoiner) {
+        return partitionByKey()
+                .joinPartition(
+                        otherStream.partitionByKey(),
+                        valueJoiner,
+                        JoinProcessor.JoinType.OUTER,
+                        JoinProcessor.JoinType.INNER);
+    }
+
+    /**
+     * Does a right outer join of the values of this stream with the values having the same key from the other stream.
+     * <p>
+     * Note: The parallelism of this stream is carried forward to the joined stream.
+     * </p>
+     *
+     * @param otherStream the other stream
+     * @param valueJoiner the {@link ValueJoiner}
+     * @param <R>         the type of the values resulting from the join
+     * @param <V1>        the type of the values in the other stream
+     * @return the new stream
+     */
+    public <R, V1> PairStream<K, R> rightOuterJoin(PairStream<K, V1> otherStream,
+                                                  ValueJoiner<? super V, ? super V1, ? extends R> valueJoiner) {
+        return partitionByKey()
+                .joinPartition(
+                        otherStream.partitionByKey(),
+                        valueJoiner,
+                        JoinProcessor.JoinType.INNER,
+                        JoinProcessor.JoinType.OUTER);
+    }
+
+    /**
+     * Does a full outer join of the values of this stream with the values having the same key from the other stream.
+     * <p>
+     * Note: The parallelism of this stream is carried forward to the joined stream.
+     * </p>
+     *
+     * @param otherStream the other stream
+     * @param valueJoiner the {@link ValueJoiner}
+     * @param <R>         the type of the values resulting from the join
+     * @param <V1>        the type of the values in the other stream
+     * @return the new stream
+     */
+    public <R, V1> PairStream<K, R> fullOuterJoin(PairStream<K, V1> otherStream,
+                                                  ValueJoiner<? super V, ? super V1, ? extends R> valueJoiner) {
+        return partitionByKey()
+                .joinPartition(
+                        otherStream.partitionByKey(),
+                        valueJoiner,
+                        JoinProcessor.JoinType.OUTER,
+                        JoinProcessor.JoinType.OUTER);
     }
 
     /**
@@ -193,7 +342,7 @@ public class PairStream<K, V> extends Stream<Pair<K, V>> {
      */
     @Override
     @SuppressWarnings("unchecked")
-    public PairStream<K, V>[] branch(Predicate<Pair<K, V>>... predicates) {
+    public PairStream<K, V>[] branch(Predicate<? super Pair<K, V>>... predicates) {
         List<PairStream<K, V>> pairStreams = new ArrayList<>();
         for (Stream<Pair<K, V>> stream : super.branch(predicates)) {
             pairStreams.add(toPairStream(stream));
@@ -202,40 +351,178 @@ public class PairStream<K, V> extends Stream<Pair<K, V>> {
     }
 
     /**
-     * Update the state by applying the given aggregator to the previous state of the
+     * Update the state by applying the given state update function to the previous state of the
      * key and the new value for the key. This internally uses {@link org.apache.storm.topology.IStatefulBolt}
      * to save the state. Use {@link Config#TOPOLOGY_STATE_PROVIDER} to choose the state implementation.
      *
-     * @param aggregator the aggregator
+     * @param stateUpdateFn the state update function
      * @param <R>        the result type
      * @return the {@link StreamState} which can be used to query the state
      */
-    public <R> StreamState<K, R> updateStateByKey(Aggregator<? super V, ? extends R> aggregator) {
+    public <R> StreamState<K, R> updateStateByKey(R initialValue,
+                                                  BiFunction<? super R, ? super V, ? extends R> stateUpdateFn) {
+        return updateStateByKey(StateUpdater.of(initialValue, stateUpdateFn));
+    }
+
+    /**
+     * Update the state by applying the given state update function to the previous state of the
+     * key and the new value for the key. This internally uses {@link org.apache.storm.topology.IStatefulBolt}
+     * to save the state. Use {@link Config#TOPOLOGY_STATE_PROVIDER} to choose the state implementation.
+     *
+     * @param stateUpdater the state updater
+     * @param <R>          the result type
+     * @return the {@link StreamState} which can be used to query the state
+     */
+    public <R> StreamState<K, R> updateStateByKey(StateUpdater<? super V, ? extends R> stateUpdater) {
+        return partitionByKey().updateStateByKeyPartition(stateUpdater);
+    }
+
+    private <R> StreamState<K, R> updateStateByKeyPartition(StateUpdater<? super V, ? extends R> stateUpdater) {
         return new StreamState<>(
-                new PairStream<>(streamBuilder, addProcessorNode(new UpdateStateByKeyProcessor<>(aggregator), KEY_VALUE)));
+                new PairStream<>(streamBuilder,
+                        addProcessorNode(new UpdateStateByKeyProcessor<>(stateUpdater), KEY_VALUE, true)));
+    }
+
+    private <R, V1> PairStream<K, R> joinPartition(PairStream<K, V1> otherStream,
+                                                   ValueJoiner<? super V, ? super V1, ? extends R> valueJoiner,
+                                                   JoinProcessor.JoinType leftType,
+                                                   JoinProcessor.JoinType rightType) {
+        String leftStream = stream;
+        String rightStream = otherStream.stream;
+        Node joinNode = addProcessorNode(
+                new JoinProcessor<>(leftStream, rightStream, valueJoiner, leftType, rightType),
+                KEY_VALUE,
+                true);
+        addNode(otherStream.getNode(), joinNode, joinNode.getParallelism());
+        return new PairStream<>(streamBuilder, joinNode);
+    }
+
+    private PairStream<K, V> partitionByKey() {
+        return shouldPartitionByKey() ? partitionBy(KEY) : this;
+    }
+
+    private boolean shouldPartitionByKey() {
+        if (node.getParallelism() == 1) {
+            return false;
+        }
+        /*
+         * if the current processor preserves the key and is
+         * already partitioned on key, skip the re-partition.
+         */
+        if (node instanceof ProcessorNode) {
+            ProcessorNode pn = (ProcessorNode) node;
+            Fields fields = pn.getGroupingInfo() == null ? null : pn.getGroupingInfo().getFields();
+            if (pn.isPreservesKey() && fields != null && fields.equals(KEY)) {
+                return false;
+            }
+        }
+        return true;
     }
 
     private PairStream<K, V> partitionBy(Fields fields) {
+        return partitionBy(fields, node.parallelism);
+    }
+
+    private PairStream<K, V> partitionBy(Fields fields, int parallelism) {
         return new PairStream<>(
                 streamBuilder,
-                addNode(new PartitionNode(stream, node.getOutputFields(), GroupingInfo.fields(fields))));
+                addNode(node, new PartitionNode(stream, node.getOutputFields(), GroupingInfo.fields(fields)), parallelism));
     }
 
     private PairStream<K, V> toPairStream(Stream<Pair<K, V>> stream) {
         return new PairStream<>(stream.streamBuilder, stream.node);
     }
 
+    private <A, R> PairStream<K, R> aggregatePartition(CombinerAggregator<? super V, A, ? extends R> aggregator) {
+        return new PairStream<>(streamBuilder,
+                addProcessorNode(new AggregateByKeyProcessor<>(aggregator), KEY_VALUE, true));
+    }
+
+    private <A> PairStream<K, A> combinePartition(CombinerAggregator<? super V, A, ?> aggregator) {
+        return new PairStream<>(streamBuilder,
+                addProcessorNode(new AggregateByKeyProcessor<>(aggregator, true), KEY_VALUE, true));
+    }
+
+    private <R> PairStream<K, R> merge(CombinerAggregator<?, V, ? extends R> aggregator) {
+        return new PairStream<>(streamBuilder,
+                addProcessorNode(new MergeAggregateByKeyProcessor<>(aggregator), KEY_VALUE, true));
+    }
+
+    private PairStream<K, V> reducePartition(Reducer<V> reducer) {
+        return new PairStream<>(streamBuilder,
+                addProcessorNode(new ReduceByKeyProcessor<>(reducer), KEY_VALUE, true));
+    }
+
+    // if re-partitioning is involved, does a per-partition aggregate by key before emitting the results downstream
+    private <A, R> PairStream<K, R> combineByKey(CombinerAggregator<? super V, A, ? extends R> aggregator) {
+        if (shouldPartitionByKey()) {
+            if (node instanceof ProcessorNode) {
+                if (node.isWindowed()) {
+                    return combinePartition(aggregator).partitionBy(KEY).merge(aggregator);
+                }
+            } else if (node instanceof WindowNode) {
+                Set<Node> parents = node.getParents();
+                Optional<Node> nonWindowed = parents.stream().filter(p -> !p.isWindowed()).findAny();
+                if (!nonWindowed.isPresent()) {
+                    parents.forEach(p -> {
+                        Node localAggregateNode = makeProcessorNode(
+                                new AggregateByKeyProcessor<>(aggregator, true), KEY_VALUE, true);
+                        streamBuilder.insert(p, localAggregateNode);
+                    });
+                    return ((PairStream<K, A>)partitionBy(KEY)).merge(aggregator);
+                }
+            }
+            return partitionBy(KEY).aggregatePartition(aggregator);
+        } else {
+            return aggregatePartition(aggregator);
+        }
+    }
+
+    // if re-partitioning is involved, does a per-partition reduce by key before emitting the results downstream
+    private PairStream<K, V> combineByKey(Reducer<V> reducer) {
+        if (shouldPartitionByKey()) {
+            if (node instanceof ProcessorNode) {
+                if (node.isWindowed()) {
+                    return reducePartition(reducer).partitionBy(KEY).reducePartition(reducer);
+                }
+            } else if (node instanceof WindowNode) {
+                for (Node p : node.getParents()) {
+                    if (p.isWindowed()) {
+                        Node localReduceNode = makeProcessorNode(new ReduceByKeyProcessor<>(reducer), KEY_VALUE, true);
+                        streamBuilder.insert(p, localReduceNode);
+                    }
+                }
+            }
+            return partitionBy(KEY).reducePartition(reducer);
+        } else {
+            return reducePartition(reducer);
+        }
+    }
+
     // used internally to merge values in groupByKeyAndWindow
-    private static class MergeValues<V> implements Aggregator<V, ArrayList<V>> {
+    private static class MergeValues<V> implements CombinerAggregator<V, ArrayList<V>, ArrayList<V>> {
         @Override
         public ArrayList<V> init() {
             return new ArrayList<>();
         }
 
         @Override
-        public ArrayList<V> apply(V value, ArrayList<V> aggregate) {
+        public ArrayList<V> apply(ArrayList<V> aggregate, V value) {
             aggregate.add(value);
             return aggregate;
         }
+
+        @Override
+        public ArrayList<V> merge(ArrayList<V> accum1, ArrayList<V> accum2) {
+            ArrayList<V> res = new ArrayList<V>();
+            res.addAll(accum1);
+            res.addAll(accum2);
+            return res;
+        }
+
+        @Override
+        public ArrayList<V> result(ArrayList<V> accum) {
+            return accum;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/PartitionNode.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/PartitionNode.java b/storm-core/src/jvm/org/apache/storm/streams/PartitionNode.java
index ca92def..90b4d7c 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/PartitionNode.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/PartitionNode.java
@@ -25,18 +25,13 @@ import org.apache.storm.tuple.Fields;
  * 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;
+        super(outputStream, outputFields, groupingInfo);
     }
 
     PartitionNode(String outputStream, Fields outputFields) {
         this(outputStream, outputFields, GroupingInfo.shuffle());
     }
 
-    GroupingInfo getGroupingInfo() {
-        return groupingInfo;
-    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java b/storm-core/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java
index 5bc6fff..3f3b5c1 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java
@@ -18,7 +18,10 @@
 package org.apache.storm.streams;
 
 import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.HashBasedTable;
 import com.google.common.collect.Multimap;
+import com.google.common.collect.Table;
+import org.apache.storm.generated.GlobalStreamId;
 import org.apache.storm.streams.processors.ChainedProcessorContext;
 import org.apache.storm.streams.processors.EmittingProcessorContext;
 import org.apache.storm.streams.processors.ForwardingProcessorContext;
@@ -55,8 +58,9 @@ class ProcessorBoltDelegate implements Serializable {
     private OutputCollector outputCollector;
     private final List<ProcessorNode> outgoingProcessors = new ArrayList<>();
     private final Set<EmittingProcessorContext> emittingProcessorContexts = new HashSet<>();
-    private final Map<ProcessorNode, Set<String>> punctuationState = new HashMap<>();
+    private final Table<ProcessorNode, String, Integer> punctuationState = HashBasedTable.create();
     private Multimap<String, ProcessorNode> streamToInitialProcessors;
+    private final Map<String, Integer> streamToInputTaskCount = new HashMap<>();
     private String timestampField;
 
     ProcessorBoltDelegate(String id, DirectedGraph<Node, Edge> graph, List<ProcessorNode> nodes) {
@@ -114,6 +118,9 @@ class ProcessorBoltDelegate implements Serializable {
                 ctx.setTimestampField(timestampField);
             }
         }
+        for (String stream : streamToInitialProcessors.keySet()) {
+            streamToInputTaskCount.put(stream, getStreamInputTaskCount(context, stream));
+        }
     }
 
     void declareOutputFields(OutputFieldsDeclarer declarer) {
@@ -127,6 +134,12 @@ class ProcessorBoltDelegate implements Serializable {
                     fields.add(timestampField);
                     declarer.declareStream(stream, new Fields(fields));
                 }
+                /*
+                 * Declare a separate 'punctuation' stream per output stream so that the receiving bolt
+                 * can subscribe to this stream with 'ALL' grouping and process the punctuation once it
+                 * receives from all upstream tasks.
+                 */
+                declarer.declareStream(StreamUtil.getPunctuationStream(stream), StreamUtil.getPunctuationFields());
             }
         }
     }
@@ -168,20 +181,32 @@ class ProcessorBoltDelegate implements Serializable {
 
     void process(Object value, String sourceStreamId) {
         LOG.debug("Process value {}, sourceStreamId {}", value, sourceStreamId);
+        if (StreamUtil.isPunctuation(value)) {
+            punctuateInitialProcessors(sourceStreamId);
+        } else {
+            executeInitialProcessors(value, sourceStreamId);
+        }
+    }
+
+    private void punctuateInitialProcessors(String punctuationStreamId) {
+        String sourceStreamId = StreamUtil.getSourceStream(punctuationStreamId);
         Collection<ProcessorNode> initialProcessors = streamToInitialProcessors.get(sourceStreamId);
         for (ProcessorNode processorNode : initialProcessors) {
-            Processor processor = processorNode.getProcessor();
-            if (StreamUtil.isPunctuation(value)) {
-                if (shouldPunctuate(processorNode, sourceStreamId)) {
-                    processor.punctuate(null);
-                    clearPunctuationState(processorNode);
-                }
-            } else {
-                processor.execute(value, sourceStreamId);
+            if (shouldPunctuate(processorNode, sourceStreamId)) {
+                processorNode.getProcessor().punctuate(null);
+                clearPunctuationState(processorNode);
             }
         }
     }
 
+    private void executeInitialProcessors(Object value, String sourceStreamId) {
+        Collection<ProcessorNode> initialProcessors = streamToInitialProcessors.get(sourceStreamId);
+        for (ProcessorNode processorNode : initialProcessors) {
+            Processor processor = processorNode.getProcessor();
+            processor.execute(value, sourceStreamId);
+        }
+    }
+
     void setStreamToInitialProcessors(Multimap<String, ProcessorNode> streamToInitialProcessors) {
         this.streamToInitialProcessors = streamToInitialProcessors;
     }
@@ -225,9 +250,6 @@ class ProcessorBoltDelegate implements Serializable {
         List<EmittingProcessorContext> emittingContexts = new ArrayList<>();
         for (String stream : processorNode.getOutputStreams()) {
             EmittingProcessorContext emittingContext = new EmittingProcessorContext(processorNode, outputCollector, stream);
-            if (StreamUtil.isSinkStream(stream)) {
-                emittingContext.setEmitPunctuation(false);
-            }
             emittingContexts.add(emittingContext);
         }
         emittingProcessorContexts.addAll(emittingContexts);
@@ -257,24 +279,48 @@ class ProcessorBoltDelegate implements Serializable {
         return children;
     }
 
-    // if we received punctuation from all parent windowed streams
+    // for the given processor node, if we received punctuation from all tasks of its parent windowed streams
     private boolean shouldPunctuate(ProcessorNode processorNode, String sourceStreamId) {
-        if (processorNode.getWindowedParentStreams().size() <= 1) {
-            return true;
+        if (!processorNode.getWindowedParentStreams().isEmpty()) {
+            updateCount(processorNode, sourceStreamId);
+            if (punctuationState.row(processorNode).size() != processorNode.getWindowedParentStreams().size()) {
+                return false;
+            }
+            // size matches, check if the streams are expected
+            Set<String> receivedStreams = punctuationState.row(processorNode).keySet();
+            if (!receivedStreams.equals(processorNode.getWindowedParentStreams())) {
+                throw new IllegalStateException("Received punctuation from streams " + receivedStreams + " expected "
+                        + processorNode.getWindowedParentStreams());
+            }
+            for (String receivedStream : receivedStreams) {
+                Integer expected = streamToInputTaskCount.get(receivedStream);
+                if (expected == null) {
+                    throw new IllegalStateException("Punctuation received on unexpected stream '" + receivedStream +
+                            "' for which input task count is not set.");
+                }
+                if (punctuationState.get(processorNode, receivedStream) < streamToInputTaskCount.get(receivedStream)) {
+                    return false;
+                }
+            }
         }
-        Set<String> receivedStreams = punctuationState.get(processorNode);
-        if (receivedStreams == null) {
-            receivedStreams = new HashSet<>();
-            punctuationState.put(processorNode, receivedStreams);
+        return true;
+    }
+
+    private void updateCount(ProcessorNode processorNode, String sourceStreamId) {
+        Integer count = punctuationState.get(processorNode, sourceStreamId);
+        if (count == null) {
+            punctuationState.put(processorNode, sourceStreamId, 1);
+        } else {
+            punctuationState.put(processorNode, sourceStreamId, count + 1);
         }
-        receivedStreams.add(sourceStreamId);
-        return receivedStreams.equals(processorNode.getWindowedParentStreams());
     }
 
     private void clearPunctuationState(ProcessorNode processorNode) {
-        Set<String> state = punctuationState.get(processorNode);
-        if (state != null) {
-            state.clear();
+        if (!punctuationState.isEmpty()) {
+            Map<String, Integer> state = punctuationState.row(processorNode);
+            if (!state.isEmpty()) {
+                state.clear();
+            }
         }
     }
 
@@ -282,4 +328,21 @@ class ProcessorBoltDelegate implements Serializable {
         return input.size() == (timestampField == null ? 2 : 3);
     }
 
+    private int getStreamInputTaskCount(TopologyContext context, String stream) {
+        int count = 0;
+        for (GlobalStreamId inputStream : context.getThisSources().keySet()) {
+            if (stream.equals(getStreamId(inputStream))) {
+                count += context.getComponentTasks(inputStream.get_componentId()).size();
+            }
+        }
+        return count;
+    }
+
+    private String getStreamId(GlobalStreamId globalStreamId) {
+        if (globalStreamId.get_componentId().startsWith("spout")) {
+            return globalStreamId.get_componentId() + globalStreamId.get_streamId();
+        }
+        return globalStreamId.get_streamId();
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/ProcessorNode.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/ProcessorNode.java b/storm-core/src/jvm/org/apache/storm/streams/ProcessorNode.java
index 4771f4f..b68dd48 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/ProcessorNode.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/ProcessorNode.java
@@ -32,32 +32,29 @@ import java.util.Set;
 public class ProcessorNode extends Node {
     private final Processor<?> processor;
     private final boolean isBatch;
-    private boolean windowed;
+    private final boolean preservesKey;
     // Windowed parent streams
     private Set<String> windowedParentStreams = Collections.emptySet();
 
-    public ProcessorNode(Processor<?> processor, String outputStream, Fields outputFields) {
+    public ProcessorNode(Processor<?> processor, String outputStream, Fields outputFields, boolean preservesKey) {
         super(outputStream, outputFields);
         this.isBatch = processor instanceof BatchProcessor;
         this.processor = processor;
+        this.preservesKey = preservesKey;
     }
 
-    public Processor<?> getProcessor() {
-        return processor;
+    public ProcessorNode(Processor<?> processor, String outputStream, Fields outputFields) {
+        this(processor, outputStream, outputFields, false);
     }
 
-    public boolean isWindowed() {
-        return windowed;
+    public Processor<?> getProcessor() {
+        return processor;
     }
 
     public boolean isBatch() {
         return isBatch;
     }
 
-    public void setWindowed(boolean windowed) {
-        this.windowed = windowed;
-    }
-
     public Set<String> getWindowedParentStreams() {
         return Collections.unmodifiableSet(windowedParentStreams);
     }
@@ -70,11 +67,16 @@ public class ProcessorNode extends Node {
         this.windowedParentStreams = new HashSet<>(windowedParentStreams);
     }
 
+    public boolean isPreservesKey() {
+        return preservesKey;
+    }
+
     @Override
     public String toString() {
         return "ProcessorNode{" +
                 "processor=" + processor +
-                ", windowed=" + windowed +
+                ", isBatch=" + isBatch +
+                ", preservesKey=" + preservesKey +
                 ", windowedParentStreams=" + windowedParentStreams +
                 "} " + super.toString();
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/Stream.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/Stream.java b/storm-core/src/jvm/org/apache/storm/streams/Stream.java
index e50e7a2..d553390 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/Stream.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/Stream.java
@@ -17,7 +17,8 @@
  */
 package org.apache.storm.streams;
 
-import org.apache.storm.streams.operations.Aggregator;
+import org.apache.storm.streams.operations.BiFunction;
+import org.apache.storm.streams.operations.CombinerAggregator;
 import org.apache.storm.streams.operations.Consumer;
 import org.apache.storm.streams.operations.FlatMapFunction;
 import org.apache.storm.streams.operations.Function;
@@ -27,12 +28,14 @@ import org.apache.storm.streams.operations.PairFunction;
 import org.apache.storm.streams.operations.Predicate;
 import org.apache.storm.streams.operations.PrintConsumer;
 import org.apache.storm.streams.operations.Reducer;
+import org.apache.storm.streams.operations.aggregators.Count;
 import org.apache.storm.streams.processors.AggregateProcessor;
 import org.apache.storm.streams.processors.BranchProcessor;
 import org.apache.storm.streams.processors.FilterProcessor;
 import org.apache.storm.streams.processors.FlatMapProcessor;
 import org.apache.storm.streams.processors.ForEachProcessor;
 import org.apache.storm.streams.processors.MapProcessor;
+import org.apache.storm.streams.processors.MergeAggregateProcessor;
 import org.apache.storm.streams.processors.PeekProcessor;
 import org.apache.storm.streams.processors.Processor;
 import org.apache.storm.streams.processors.ReduceProcessor;
@@ -42,9 +45,13 @@ import org.apache.storm.topology.IBasicBolt;
 import org.apache.storm.topology.IRichBolt;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Optional;
+import java.util.Set;
 
 /**
  * Represents a stream of values.
@@ -52,6 +59,8 @@ import java.util.List;
  * @param <T> the type of the value
  */
 public class Stream<T> {
+    private static final Logger LOG = LoggerFactory.getLogger(Stream.class);
+
     protected static final Fields KEY = new Fields("key");
     protected static final Fields VALUE = new Fields("value");
     protected static final Fields KEY_VALUE = new Fields("key", "value");
@@ -79,7 +88,7 @@ public class Stream<T> {
      * @return the new stream
      */
     public Stream<T> filter(Predicate<? super T> predicate) {
-        return new Stream<>(streamBuilder, addProcessorNode(new FilterProcessor<>(predicate), VALUE));
+        return new Stream<>(streamBuilder, addProcessorNode(new FilterProcessor<>(predicate), VALUE, true));
     }
 
     /**
@@ -100,7 +109,7 @@ public class Stream<T> {
      * @param <V>      the value type
      * @return the new stream of key-value pairs
      */
-    public <K, V> PairStream<K, V> mapToPair(PairFunction<T, K, V> function) {
+    public <K, V> PairStream<K, V> mapToPair(PairFunction<? super T, ? extends K, ? extends V> function) {
         return new PairStream<>(streamBuilder, addProcessorNode(new MapProcessor<>(function), KEY_VALUE));
     }
 
@@ -113,7 +122,7 @@ public class Stream<T> {
      * @param function a mapping function to be applied to each value in this stream which produces new values.
      * @return the new stream
      */
-    public <R> Stream<R> flatMap(FlatMapFunction<T, R> function) {
+    public <R> Stream<R> flatMap(FlatMapFunction<? super T, ? extends R> function) {
         return new Stream<>(streamBuilder, addProcessorNode(new FlatMapProcessor<>(function), VALUE));
     }
 
@@ -128,7 +137,7 @@ public class Stream<T> {
      * @see #flatMap(FlatMapFunction)
      * @see #mapToPair(PairFunction)
      */
-    public <K, V> PairStream<K, V> flatMapToPair(PairFlatMapFunction<T, K, V> function) {
+    public <K, V> PairStream<K, V> flatMapToPair(PairFlatMapFunction<? super T, ? extends K, ? extends V> function) {
         return new PairStream<>(streamBuilder, addProcessorNode(new FlatMapProcessor<>(function), KEY_VALUE));
     }
 
@@ -173,12 +182,11 @@ public class Stream<T> {
      * @return the new stream
      */
     public Stream<T> peek(Consumer<? super T> action) {
-        return new Stream<>(streamBuilder, addProcessorNode(new PeekProcessor<>(action), node.getOutputFields()));
+        return new Stream<>(streamBuilder, addProcessorNode(new PeekProcessor<>(action), node.getOutputFields(), true));
     }
 
     /**
-     * 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.
+     * Aggregates the values in this stream using the aggregator. This does a global aggregation of values across all partitions.
      * <p>
      * 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.
@@ -186,15 +194,52 @@ public class Stream<T> {
      * </p>
      *
      * @param aggregator the aggregator
+     * @param <A>        the accumulator type
      * @param <R>        the result type
      * @return the new stream
      */
-    public <R> Stream<R> aggregate(Aggregator<? super T, ? extends R> aggregator) {
-        return new Stream<>(streamBuilder, global().addProcessorNode(new AggregateProcessor<>(aggregator), VALUE));
+    public <A, R> Stream<R> aggregate(CombinerAggregator<? super T, A, ? extends R> aggregator) {
+        return combine(aggregator);
+    }
+
+    /**
+     * Aggregates the values in this stream using the given initial value, accumulator and combiner. This does a global
+     * aggregation of values across all partitions.
+     * <p>
+     * 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.
+     * </p>
+     *
+     * @param initialValue the initial value of the result
+     * @param accumulator  the accumulator
+     * @param combiner     the combiner
+     * @param <R>          the result type
+     * @return the new stream
+     */
+    public <R> Stream<R> aggregate(R initialValue,
+                                   BiFunction<? super R, ? super T, ? extends R> accumulator,
+                                   BiFunction<? super R, ? super R, ? extends R> combiner) {
+        return combine(CombinerAggregator.of(initialValue, accumulator, combiner));
+    }
+
+    /**
+     * Counts the number of values in this stream. This does a global count of values across all partitions.
+     * <p>
+     * If the stream is windowed, the counts are emitted after each window activation and represents the
+     * count of elements that fall within that window.
+     * If the stream is not windowed, the count is emitted as each new element in the stream is processed.
+     * </p>
+     *
+     * @return the new stream
+     */
+    public Stream<Long> count() {
+        return aggregate(new Count<>());
     }
 
     /**
      * Performs a reduction on the elements of this stream, by repeatedly applying the reducer.
+     * This does a global reduction of values across all partitions.
      * <p>
      * If the stream is windowed, the result is emitted after each window activation and represents the
      * reduction of elements that fall within that window.
@@ -205,7 +250,7 @@ public class Stream<T> {
      * @return the new stream
      */
     public Stream<T> reduce(Reducer<T> reducer) {
-        return new Stream<>(streamBuilder, global().addProcessorNode(new ReduceProcessor<>(reducer), VALUE));
+        return combine(reducer);
     }
 
     /**
@@ -219,6 +264,10 @@ public class Stream<T> {
         if (parallelism < 1) {
             throw new IllegalArgumentException("Parallelism should be >= 1");
         }
+        if (node.getParallelism() == parallelism) {
+            LOG.debug("Node's current parallelism {}, new parallelism {}", node.getParallelism(), parallelism);
+            return this;
+        }
         Node partitionNode = addNode(node, new PartitionNode(stream, node.getOutputFields()), parallelism);
         return new Stream<>(streamBuilder, partitionNode);
     }
@@ -233,12 +282,12 @@ public class Stream<T> {
      * @return an array of result streams (branches) corresponding to the given predicates
      */
     @SuppressWarnings("unchecked")
-    public Stream<T>[] branch(Predicate<T>... predicates) {
+    public Stream<T>[] branch(Predicate<? super T>... predicates) {
         List<Stream<T>> childStreams = new ArrayList<>();
         if (predicates.length > 0) {
             BranchProcessor<T> branchProcessor = new BranchProcessor<>();
             Node branchNode = addProcessorNode(branchProcessor, VALUE);
-            for (Predicate<T> predicate : predicates) {
+            for (Predicate<? super T> 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";
@@ -321,10 +370,8 @@ public class Stream<T> {
      * @return the result stream
      */
     public <V> PairStream<T, V> stateQuery(StreamState<T, V> streamState) {
-        // need all grouping for state query since the state is local
+        // need all grouping for state query since the state is per-task
         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);
     }
 
@@ -337,13 +384,17 @@ public class Stream<T> {
     }
 
     Node addNode(Node child) {
-        return addNode(this.node, child);
+        return addNode(node, child);
     }
 
     Node addProcessorNode(Processor<?> processor, Fields outputFields) {
         return addNode(makeProcessorNode(processor, outputFields));
     }
 
+    Node addProcessorNode(Processor<?> processor, Fields outputFields, boolean preservesKey) {
+        return addNode(makeProcessorNode(processor, outputFields, preservesKey));
+    }
+
     String getStream() {
         return stream;
     }
@@ -356,12 +407,16 @@ public class Stream<T> {
         return streamBuilder.addNode(parent, child, parentStreamId);
     }
 
-    private Node addNode(Node child, int parallelism, String parentStreamId) {
-        return streamBuilder.addNode(this.node, child, parallelism, parentStreamId);
+    private Node addNode(Node parent, Node child, String parentStreamId, int parallelism) {
+        return streamBuilder.addNode(parent, child, parentStreamId, parallelism);
     }
 
     private ProcessorNode makeProcessorNode(Processor<?> processor, Fields outputFields) {
-        return new ProcessorNode(processor, UniqueIdGen.getInstance().getUniqueStreamId(), outputFields);
+        return makeProcessorNode(processor, outputFields, false);
+    }
+
+    ProcessorNode makeProcessorNode(Processor<?> processor, Fields outputFields, boolean preservesKey) {
+        return new ProcessorNode(processor, UniqueIdGen.getInstance().getUniqueStreamId(), outputFields, preservesKey);
     }
 
     private void addSinkNode(SinkNode sinkNode, int parallelism) {
@@ -369,15 +424,9 @@ public class Stream<T> {
         sinkNode.setComponentId(boltId);
         sinkNode.setParallelism(parallelism);
         if (node instanceof SpoutNode) {
-            addNode(sinkNode, parallelism, Utils.DEFAULT_STREAM_ID);
+            addNode(node, sinkNode, Utils.DEFAULT_STREAM_ID, parallelism);
         } 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);
+            addNode(node, sinkNode, parallelism);
         }
     }
 
@@ -387,7 +436,79 @@ public class Stream<T> {
     }
 
     private Stream<T> all() {
+        if (node.getParallelism() == 1) {
+            return this;
+        }
         Node partitionNode = addNode(new PartitionNode(stream, node.getOutputFields(), GroupingInfo.all()));
         return new Stream<>(streamBuilder, partitionNode);
     }
+
+    private boolean shouldPartition() {
+        return node.getParallelism() > 1;
+    }
+
+    private <A> Stream<A> combinePartition(CombinerAggregator<? super T, A, ?> aggregator) {
+        return new Stream<>(streamBuilder,
+                addProcessorNode(new AggregateProcessor<>(aggregator, true), VALUE, true));
+    }
+
+    private <R> Stream<R> merge(CombinerAggregator<?, T, ? extends R> aggregator) {
+        return new Stream<>(streamBuilder,
+                addProcessorNode(new MergeAggregateProcessor<>(aggregator), VALUE));
+    }
+
+    private  <A, R> Stream<R> aggregatePartition(CombinerAggregator<? super T, A, ? extends R> aggregator) {
+        return new Stream<>(streamBuilder, addProcessorNode(new AggregateProcessor<>(aggregator), VALUE));
+    }
+
+    private Stream<T> reducePartition(Reducer<T> reducer) {
+        return new Stream<>(streamBuilder, addProcessorNode(new ReduceProcessor<>(reducer), VALUE));
+    }
+
+    // if re-partitioning is involved, does a per-partition aggregate before emitting the results downstream
+    private <A, R> Stream<R> combine(CombinerAggregator<? super T, A, ? extends R> aggregator) {
+        if (shouldPartition()) {
+            if (node instanceof ProcessorNode) {
+                if (node.isWindowed()) {
+                    return combinePartition(aggregator).global().merge(aggregator);
+                }
+            } else if (node instanceof WindowNode) {
+                Set<Node> parents = node.getParents();
+                Optional<Node> nonWindowed = parents.stream().filter(p -> !p.isWindowed()).findAny();
+                if (!nonWindowed.isPresent()) {
+                    parents.forEach(p -> {
+                        Node localAggregateNode = makeProcessorNode(
+                                new AggregateProcessor<>(aggregator, true), VALUE, true);
+                        streamBuilder.insert(p, localAggregateNode);
+                    });
+                    return ((Stream<A>)global()).merge(aggregator);
+                }
+            }
+            return global().aggregatePartition(aggregator);
+        } else {
+            return aggregatePartition(aggregator);
+        }
+    }
+
+    // if re-partitioning is involved, does a per-partition reduce before emitting the results downstream
+    private Stream<T> combine(Reducer<T> reducer) {
+        if (shouldPartition()) {
+            if (node instanceof ProcessorNode) {
+                if (node.isWindowed()) {
+                    return reducePartition(reducer).global().reducePartition(reducer);
+                }
+            } else if (node instanceof WindowNode) {
+                for (Node p : node.getParents()) {
+                    if (p.isWindowed()) {
+                        Node localReduceNode = makeProcessorNode(new ReduceProcessor<>(reducer), VALUE);
+                        streamBuilder.insert(p, localReduceNode);
+                    }
+                }
+            }
+            return global().reducePartition(reducer);
+        } else {
+            return reducePartition(reducer);
+        }
+    }
+
 }


[06/12] storm git commit: [STORM-1961] A few fixes and refactoring

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple9.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple9.java b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple9.java
new file mode 100644
index 0000000..0cd7139
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple9.java
@@ -0,0 +1,106 @@
+/**
+ * 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.tuple;
+
+/**
+ * A tuple of nine elements along the lines of Scala's Tuple.
+ *
+ * @param <T1> the type of the first element
+ * @param <T2> the type of the second element
+ * @param <T3> the type of the third element
+ * @param <T4> the type of the fourth element
+ * @param <T5> the type of the fifth element
+ * @param <T6> the type of the sixth element
+ * @param <T7> the type of the seventh element
+ * @param <T8> the type of the eighth element
+ * @param <T9> the type of the ninth element
+ */
+public class Tuple9<T1, T2, T3, T4, T5, T6, T7, T8, T9> {
+    public final T1 _1;
+    public final T2 _2;
+    public final T3 _3;
+    public final T4 _4;
+    public final T5 _5;
+    public final T6 _6;
+    public final T7 _7;
+    public final T8 _8;
+    public final T9 _9;
+
+    /**
+     * Constructs a new tuple.
+     *
+     * @param _1 the first element
+     * @param _2 the second element
+     * @param _3 the third element
+     * @param _4 the fourth element
+     * @param _5 the fifth element
+     * @param _6 the sixth element
+     * @param _7 the seventh element
+     * @param _8 the eighth element
+     * @param _9 the ninth element
+     */
+    public Tuple9(T1 _1, T2 _2, T3 _3, T4 _4, T5 _5, T6 _6, T7 _7, T8 _8, T9 _9) {
+        this._1 = _1;
+        this._2 = _2;
+        this._3 = _3;
+        this._4 = _4;
+        this._5 = _5;
+        this._6 = _6;
+        this._7 = _7;
+        this._8 = _8;
+        this._9 = _9;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        Tuple9<?, ?, ?, ?, ?, ?, ?, ?, ?> tuple9 = (Tuple9<?, ?, ?, ?, ?, ?, ?, ?, ?>) o;
+
+        if (_1 != null ? !_1.equals(tuple9._1) : tuple9._1 != null) return false;
+        if (_2 != null ? !_2.equals(tuple9._2) : tuple9._2 != null) return false;
+        if (_3 != null ? !_3.equals(tuple9._3) : tuple9._3 != null) return false;
+        if (_4 != null ? !_4.equals(tuple9._4) : tuple9._4 != null) return false;
+        if (_5 != null ? !_5.equals(tuple9._5) : tuple9._5 != null) return false;
+        if (_6 != null ? !_6.equals(tuple9._6) : tuple9._6 != null) return false;
+        if (_7 != null ? !_7.equals(tuple9._7) : tuple9._7 != null) return false;
+        if (_8 != null ? !_8.equals(tuple9._8) : tuple9._8 != null) return false;
+        return _9 != null ? _9.equals(tuple9._9) : tuple9._9 == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = _1 != null ? _1.hashCode() : 0;
+        result = 31 * result + (_2 != null ? _2.hashCode() : 0);
+        result = 31 * result + (_3 != null ? _3.hashCode() : 0);
+        result = 31 * result + (_4 != null ? _4.hashCode() : 0);
+        result = 31 * result + (_5 != null ? _5.hashCode() : 0);
+        result = 31 * result + (_6 != null ? _6.hashCode() : 0);
+        result = 31 * result + (_7 != null ? _7.hashCode() : 0);
+        result = 31 * result + (_8 != null ? _8.hashCode() : 0);
+        result = 31 * result + (_9 != null ? _9.hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "(" + _1 + "," + _2 + "," + _3 + "," + _4 + "," + _5 + "," + _6 + "," + _7 + "," + _8 + "," + _9 + ")";
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java b/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java
index e9d5127..aa877f9 100644
--- a/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java
+++ b/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java
@@ -18,6 +18,7 @@
 package org.apache.storm.streams;
 
 import com.google.common.collect.Multimap;
+import org.apache.storm.generated.GlobalStreamId;
 import org.apache.storm.streams.operations.aggregators.Sum;
 import org.apache.storm.streams.processors.AggregateProcessor;
 import org.apache.storm.streams.processors.FilterProcessor;
@@ -34,10 +35,10 @@ import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 import org.mockito.Mockito;
 
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.Map;
 import java.util.Set;
 
 import static org.junit.Assert.assertArrayEquals;
@@ -99,7 +100,7 @@ public class ProcessorBoltTest {
         assertArrayEquals(new Object[]{mockTuple2, mockTuple3, punctuation}, anchor.getAllValues().get(0).toArray());
         assertArrayEquals(new Object[]{mockTuple2, mockTuple3, punctuation}, anchor.getAllValues().get(1).toArray());
         assertArrayEquals(new Object[]{new Values(200L), new Values("__punctuation")}, values.getAllValues().toArray());
-        assertArrayEquals(new Object[]{"outputstream", "outputstream"}, os.getAllValues().toArray());
+        assertArrayEquals(new Object[]{"outputstream", "outputstream__punctuation"}, os.getAllValues().toArray());
         Mockito.verify(mockOutputCollector).ack(mockTuple2);
         Mockito.verify(mockOutputCollector).ack(mockTuple3);
         Mockito.verify(mockOutputCollector).ack(punctuation);
@@ -138,6 +139,14 @@ public class ProcessorBoltTest {
         node.setWindowedParentStreams(windowedParentStreams);
         node.setWindowed(isWindowed);
         Mockito.when(mockStreamToProcessors.get(Mockito.anyString())).thenReturn(Collections.singletonList(node));
+        Mockito.when(mockStreamToProcessors.keySet()).thenReturn(Collections.singleton("inputstream"));
+        Map mockSources = Mockito.mock(Map.class);
+        GlobalStreamId mockGlobalStreamId = Mockito.mock(GlobalStreamId.class);
+        Mockito.when(mockTopologyContext.getThisSources()).thenReturn(mockSources);
+        Mockito.when(mockSources.keySet()).thenReturn(Collections.singleton(mockGlobalStreamId));
+        Mockito.when(mockGlobalStreamId.get_streamId()).thenReturn("inputstream");
+        Mockito.when(mockGlobalStreamId.get_componentId()).thenReturn("bolt0");
+        Mockito.when(mockTopologyContext.getComponentTasks(Mockito.anyString())).thenReturn(Collections.singletonList(1));
         graph.addVertex(node);
         bolt = new ProcessorBolt("bolt1", graph, Collections.singletonList(node));
         if (tsFieldName != null && !tsFieldName.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/test/jvm/org/apache/storm/streams/StatefulProcessorBoltTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/streams/StatefulProcessorBoltTest.java b/storm-core/test/jvm/org/apache/storm/streams/StatefulProcessorBoltTest.java
index dbc7e27..2e6e54a 100644
--- a/storm-core/test/jvm/org/apache/storm/streams/StatefulProcessorBoltTest.java
+++ b/storm-core/test/jvm/org/apache/storm/streams/StatefulProcessorBoltTest.java
@@ -19,7 +19,7 @@ package org.apache.storm.streams;
 
 import com.google.common.collect.Multimap;
 import org.apache.storm.state.KeyValueState;
-import org.apache.storm.streams.operations.aggregators.Count;
+import org.apache.storm.streams.operations.StateUpdater;
 import org.apache.storm.streams.processors.Processor;
 import org.apache.storm.streams.processors.UpdateStateByKeyProcessor;
 import org.apache.storm.task.OutputCollector;
@@ -65,7 +65,17 @@ public class StatefulProcessorBoltTest {
 
     @Test
     public void testEmitAndAck() throws Exception {
-        setUpStatefulProcessorBolt(new UpdateStateByKeyProcessor<>(new Count<>()));
+        setUpStatefulProcessorBolt(new UpdateStateByKeyProcessor<>(new StateUpdater<Object, Long>() {
+            @Override
+            public Long init() {
+                return 0L;
+            }
+
+            @Override
+            public Long apply(Long state, Object value) {
+                return state + 1;
+            }
+        }));
         bolt.execute(mockTuple1);
         ArgumentCaptor<Collection> anchor = ArgumentCaptor.forClass(Collection.class);
         ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
@@ -80,6 +90,7 @@ public class StatefulProcessorBoltTest {
 
     private void setUpStatefulProcessorBolt(Processor<?> processor) {
         ProcessorNode node = new ProcessorNode(processor, "outputstream", new Fields("value"));
+        node.setEmitsPair(true);
         Mockito.when(mockStreamToProcessors.get(Mockito.anyString())).thenReturn(Collections.singletonList(node));
         graph = new DefaultDirectedGraph(new StreamsEdgeFactory());
         graph.addVertex(node);

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/test/jvm/org/apache/storm/streams/StreamBuilderTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/streams/StreamBuilderTest.java b/storm-core/test/jvm/org/apache/storm/streams/StreamBuilderTest.java
index 1498ae4..c88fe34 100644
--- a/storm-core/test/jvm/org/apache/storm/streams/StreamBuilderTest.java
+++ b/storm-core/test/jvm/org/apache/storm/streams/StreamBuilderTest.java
@@ -28,6 +28,7 @@ import org.apache.storm.streams.operations.aggregators.Count;
 import org.apache.storm.streams.operations.mappers.PairValueMapper;
 import org.apache.storm.streams.operations.mappers.ValueMapper;
 import org.apache.storm.streams.processors.BranchProcessor;
+import org.apache.storm.streams.windowing.TumblingWindows;
 import org.apache.storm.task.OutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.topology.IRichBolt;
@@ -35,6 +36,7 @@ import org.apache.storm.topology.IRichSpout;
 import org.apache.storm.topology.OutputFieldsDeclarer;
 import org.apache.storm.topology.base.BaseRichBolt;
 import org.apache.storm.topology.base.BaseRichSpout;
+import org.apache.storm.topology.base.BaseWindowedBolt;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.utils.Utils;
@@ -115,9 +117,9 @@ public class StreamBuilderTest {
 
     @Test
     public void testGroupBy() throws Exception {
-        PairStream<String, String> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new PairValueMapper<>(0, 1));
+        PairStream<String, String> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new PairValueMapper<>(0, 1), 2);
 
-        stream.groupByKey().aggregateByKey(new Count<>());
+        stream.window(TumblingWindows.of(BaseWindowedBolt.Count.of(10))).aggregateByKey(new Count<>());
 
         StormTopology topology = streamBuilder.build();
         assertEquals(2, topology.get_bolts_size());
@@ -129,7 +131,7 @@ public class StreamBuilderTest {
 
     @Test
     public void testGlobalAggregate() throws Exception {
-        Stream<String> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0));
+        Stream<String> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0), 2);
 
         stream.aggregate(new Count<>());
 
@@ -142,6 +144,7 @@ public class StreamBuilderTest {
         expected1.put(new GlobalStreamId(spoutId, "default"), Grouping.shuffle(new NullStruct()));
         Map<GlobalStreamId, Grouping> expected2 = new HashMap<>();
         expected2.put(new GlobalStreamId("bolt1", "s1"), Grouping.fields(Collections.emptyList()));
+        expected2.put(new GlobalStreamId("bolt1", "s1__punctuation"), Grouping.all(new NullStruct()));
         assertEquals(expected1, bolt1.get_common().get_inputs());
         assertEquals(expected2, bolt2.get_common().get_inputs());
     }
@@ -169,11 +172,57 @@ public class StreamBuilderTest {
     public void testBranchAndJoin() throws Exception {
         TopologyContext mockContext = Mockito.mock(TopologyContext.class);
         OutputCollector mockCollector = Mockito.mock(OutputCollector.class);
-        Stream<Integer> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0));
+        Stream<Integer> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0), 2);
         Stream<Integer>[] streams = stream.branch(x -> x % 2 == 0, x -> x % 2 == 1);
         PairStream<Integer, Pair<Integer, Integer>> joined = streams[0].mapToPair(x -> Pair.of(x, 1)).join(streams[1].mapToPair(x -> Pair.of(x, 1)));
         assertTrue(joined.getNode() instanceof ProcessorNode);
         StormTopology topology = streamBuilder.build();
+        assertEquals(2, topology.get_bolts_size());
+    }
+
+    @Test
+    public void testMultiPartitionByKey() {
+        TopologyContext mockContext = Mockito.mock(TopologyContext.class);
+        OutputCollector mockCollector = Mockito.mock(OutputCollector.class);
+        Stream<Integer> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0));
+        stream.mapToPair(x -> Pair.of(x, x))
+                .window(TumblingWindows.of(BaseWindowedBolt.Count.of(10)))
+                .reduceByKey((x, y) -> x + y)
+                .reduceByKey((x, y) -> 0)
+                .print();
+        StormTopology topology = streamBuilder.build();
+        assertEquals(2, topology.get_bolts_size());
+    }
+
+    @Test
+    public void testMultiPartitionByKeyWithRepartition() {
+        TopologyContext mockContext = Mockito.mock(TopologyContext.class);
+        OutputCollector mockCollector = Mockito.mock(OutputCollector.class);
+        Map<GlobalStreamId, Grouping> expected = new HashMap<>();
+        expected.put(new GlobalStreamId("bolt2", "s3"), Grouping.fields(Collections.singletonList("key")));
+        expected.put(new GlobalStreamId("bolt2", "s3__punctuation"), Grouping.all(new NullStruct()));
+        Stream<Integer> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0));
+        stream.mapToPair(x -> Pair.of(x, x))
+                .window(TumblingWindows.of(BaseWindowedBolt.Count.of(10)))
+                .reduceByKey((x, y) -> x + y)
+                .repartition(10)
+                .reduceByKey((x, y) -> 0)
+                .print();
+        StormTopology topology = streamBuilder.build();
+        assertEquals(3, topology.get_bolts_size());
+        assertEquals(expected, topology.get_bolts().get("bolt3").get_common().get_inputs());
+
+    }
+
+    @Test
+    public void testPartitionByKeySinglePartition() {
+        TopologyContext mockContext = Mockito.mock(TopologyContext.class);
+        OutputCollector mockCollector = Mockito.mock(OutputCollector.class);
+        Stream<Integer> stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0));
+        stream.mapToPair(x -> Pair.of(x, x))
+                .reduceByKey((x, y) -> x + y)
+                .print();
+        StormTopology topology = streamBuilder.build();
         assertEquals(1, topology.get_bolts_size());
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/test/jvm/org/apache/storm/streams/WindowedProcessorBoltTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/streams/WindowedProcessorBoltTest.java b/storm-core/test/jvm/org/apache/storm/streams/WindowedProcessorBoltTest.java
index 7428e3f..1010f6f 100644
--- a/storm-core/test/jvm/org/apache/storm/streams/WindowedProcessorBoltTest.java
+++ b/storm-core/test/jvm/org/apache/storm/streams/WindowedProcessorBoltTest.java
@@ -77,7 +77,7 @@ public class WindowedProcessorBoltTest {
         Mockito.verify(mockOutputCollector, Mockito.times(2)).emit(os.capture(), values.capture());
         assertEquals("outputstream", os.getAllValues().get(0));
         assertEquals(new Values(3L), values.getAllValues().get(0));
-        assertEquals("outputstream", os.getAllValues().get(1));
+        assertEquals("outputstream__punctuation", os.getAllValues().get(1));
         assertEquals(new Values(WindowNode.PUNCTUATION), values.getAllValues().get(1));
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/test/jvm/org/apache/storm/streams/processors/JoinProcessorTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/streams/processors/JoinProcessorTest.java b/storm-core/test/jvm/org/apache/storm/streams/processors/JoinProcessorTest.java
new file mode 100644
index 0000000..a8ace8a
--- /dev/null
+++ b/storm-core/test/jvm/org/apache/storm/streams/processors/JoinProcessorTest.java
@@ -0,0 +1,108 @@
+package org.apache.storm.streams.processors;
+
+import org.apache.storm.streams.Pair;
+import org.apache.storm.streams.operations.PairValueJoiner;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+import static org.junit.Assert.*;
+
+public class JoinProcessorTest {
+    JoinProcessor<Integer, Pair<Integer, Integer>, Integer, Integer> joinProcessor;
+    String leftStream = "left";
+    String rightStream = "right";
+    List<Pair<Integer, List<Pair<Integer, Integer>>>> res = new ArrayList<>();
+
+    ProcessorContext context = new ProcessorContext() {
+        @Override
+        public <T> void forward(T input) {
+            res.add((Pair<Integer, List<Pair<Integer, Integer>>>)input);
+        }
+
+        @Override
+        public <T> void forward(T input, String stream) {
+        }
+
+        @Override
+        public boolean isWindowed() {
+            return true;
+        }
+
+        @Override
+        public Set<String> getWindowedParentStreams() {
+            return null;
+        }
+    };
+
+    List<Pair<Integer, Integer>> leftKeyValeus = Arrays.asList(
+            Pair.of(2, 4),
+            Pair.of(5, 25),
+            Pair.of(7, 49)
+    );
+
+    List<Pair<Integer, Integer>> rightKeyValues = Arrays.asList(
+            Pair.of(1, 1),
+            Pair.of(2, 8),
+            Pair.of(5, 125),
+            Pair.of(6, 216)
+    );
+
+    @Test
+    public void testInnerJoin() throws Exception {
+        joinProcessor = new JoinProcessor<>(leftStream, rightStream, new PairValueJoiner<>());
+        processValues();
+        assertEquals(Pair.of(2, Pair.of(4, 8)), res.get(0));
+        assertEquals(Pair.of(5, Pair.of(25, 125)), res.get(1));
+    }
+
+    @Test
+    public void testLeftOuterJoin() throws Exception {
+        joinProcessor = new JoinProcessor<>(leftStream, rightStream, new PairValueJoiner<>(),
+                JoinProcessor.JoinType.OUTER, JoinProcessor.JoinType.INNER);
+        processValues();
+        assertEquals(Pair.of(2, Pair.of(4, 8)), res.get(0));
+        assertEquals(Pair.of(5, Pair.of(25, 125)), res.get(1));
+        assertEquals(Pair.of(7, Pair.of(49, null)), res.get(2));
+    }
+
+    @Test
+    public void testRightOuterJoin() throws Exception {
+        joinProcessor = new JoinProcessor<>(leftStream, rightStream, new PairValueJoiner<>(),
+                JoinProcessor.JoinType.INNER, JoinProcessor.JoinType.OUTER);
+        processValues();
+        assertEquals(Pair.of(1, Pair.of(null, 1)), res.get(0));
+        assertEquals(Pair.of(2, Pair.of(4, 8)), res.get(1));
+        assertEquals(Pair.of(5, Pair.of(25, 125)), res.get(2));
+        assertEquals(Pair.of(6, Pair.of(null, 216)), res.get(3));
+    }
+
+    @Test
+    public void testFullOuterJoin() throws Exception {
+        joinProcessor = new JoinProcessor<>(leftStream, rightStream, new PairValueJoiner<>(),
+                JoinProcessor.JoinType.OUTER, JoinProcessor.JoinType.OUTER);
+        processValues();
+        assertEquals(Pair.of(1, Pair.of(null, 1)), res.get(0));
+        assertEquals(Pair.of(2, Pair.of(4, 8)), res.get(1));
+        assertEquals(Pair.of(5, Pair.of(25, 125)), res.get(2));
+        assertEquals(Pair.of(6, Pair.of(null, 216)), res.get(3));
+        assertEquals(Pair.of(7, Pair.of(49, null)), res.get(4));
+    }
+
+    private void processValues() {
+        res.clear();
+        joinProcessor.init(context);
+        for (Pair<Integer, Integer> kv : leftKeyValeus) {
+            joinProcessor.execute(kv, leftStream);
+        }
+        for (Pair<Integer, Integer> kv : rightKeyValues) {
+            joinProcessor.execute(kv, rightStream);
+        }
+        joinProcessor.finish();
+    }
+
+}
\ No newline at end of file


[02/12] storm git commit: [STORM-1961] Stream api for storm core use cases

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/operations/ValueJoiner.java
----------------------------------------------------------------------
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 0000000..003d097
--- /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 <V1> the type of the first value
+ * @param <V2> the type of the second value
+ * @param <R>  the result type
+ */
+public interface ValueJoiner<V1, V2, R> 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);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Count.java
----------------------------------------------------------------------
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 0000000..fd02d2a
--- /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 <T> the value type
+ */
+public class Count<T> implements Aggregator<T, Long> {
+    @Override
+    public Long init() {
+        return 0L;
+    }
+
+    @Override
+    public Long apply(T value, Long aggregate) {
+        return aggregate + 1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Sum.java
----------------------------------------------------------------------
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 0000000..e232075
--- /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<Number, Long> {
+    @Override
+    public Long init() {
+        return 0L;
+    }
+
+    @Override
+    public Long apply(Number value, Long aggregate) {
+        return value.longValue() + aggregate;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/PairValueMapper.java
----------------------------------------------------------------------
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 0000000..1490e51
--- /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 <K> the key type
+ * @param <V> the value type
+ */
+public class PairValueMapper<K, V> implements TupleValueMapper<Pair<K, V>>, PairFunction<Tuple, K, V> {
+    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<K, V> apply(Tuple input) {
+        return Pair.of((K) input.getValue(keyIndex), (V) input.getValue(valueIndex));
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/TupleValueMapper.java
----------------------------------------------------------------------
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 0000000..6ffbd88
--- /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 <T> the result type
+ */
+public interface TupleValueMapper<T> extends Function<Tuple, T> {
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/ValueMapper.java
----------------------------------------------------------------------
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 0000000..0468d0d
--- /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 <T> the value type
+ */
+public class ValueMapper<T> implements TupleValueMapper<T> {
+    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);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/ValuesMapper.java
----------------------------------------------------------------------
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 0000000..f1d8e3b
--- /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<Values> {
+    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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java
----------------------------------------------------------------------
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 0000000..d53a52d
--- /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<K, V, R> extends BaseProcessor<Pair<K, V>> implements BatchProcessor {
+    private final Aggregator<V, R> aggregator;
+    private final Map<K, R> state = new HashMap<>();
+
+    public AggregateByKeyProcessor(Aggregator<V, R> aggregator) {
+        this.aggregator = aggregator;
+    }
+
+    @Override
+    public void execute(Pair<K, V> 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<K, R> entry : state.entrySet()) {
+            context.forward(Pair.of(entry.getKey(), entry.getValue()));
+        }
+        state.clear();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java
----------------------------------------------------------------------
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 0000000..c5a1906
--- /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<T, R> extends BaseProcessor<T> implements BatchProcessor {
+    private final Aggregator<T, R> aggregator;
+    private R state;
+
+    public AggregateProcessor(Aggregator<T, R> 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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/BaseProcessor.java
----------------------------------------------------------------------
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 0000000..3ea469c
--- /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 <T>
+ */
+abstract class BaseProcessor<T> implements Processor<T> {
+    private final Set<String> 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 <R>    the result type
+     */
+    protected final <R> void mayBeForwardAggUpdate(R result) {
+        if (!context.isWindowed()) {
+            context.forward(result);
+        }
+    }
+
+    private boolean shouldPunctuate(String parentStream) {
+        punctuationState.add(parentStream);
+        return punctuationState.equals(context.getWindowedParentStreams());
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/BatchProcessor.java
----------------------------------------------------------------------
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 0000000..f9a39b4
--- /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 {
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/BranchProcessor.java
----------------------------------------------------------------------
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 0000000..f8bc739
--- /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<T> extends BaseProcessor<T> {
+    private final Map<Predicate<T>, String> predicateToStream = new HashMap<>();
+
+    public void addPredicate(Predicate<T> predicate, String stream) {
+        predicateToStream.put(predicate, stream);
+    }
+
+    @Override
+    public void execute(T input) {
+        for (Map.Entry<Predicate<T>, String> entry : predicateToStream.entrySet()) {
+            if (entry.getKey().test(input)) {
+                context.forward(input, entry.getValue());
+                break;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/ChainedProcessorContext.java
----------------------------------------------------------------------
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 0000000..c92d992
--- /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 <T> void forward(T input) {
+        for (ProcessorContext context : contexts) {
+            context.forward(input);
+        }
+    }
+
+    @Override
+    public <T> void forward(T input, String stream) {
+        for (ProcessorContext context : contexts) {
+            context.forward(input, stream);
+        }
+    }
+
+    @Override
+    public boolean isWindowed() {
+        return processorNode.isWindowed();
+    }
+
+    @Override
+    public Set<String> getWindowedParentStreams() {
+        return processorNode.getWindowedParentStreams();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/EmittingProcessorContext.java
----------------------------------------------------------------------
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 0000000..d841080
--- /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<RefCountedTuple> 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 <T> 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 <T> void forward(T input, String stream) {
+        if (stream.equals(outputStreamId)) {
+            forward(input);
+        }
+    }
+
+    @Override
+    public boolean isWindowed() {
+        return processorNode.isWindowed();
+    }
+
+    @Override
+    public Set<String> 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<Tuple> tuples(Collection<RefCountedTuple> 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);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/FilterProcessor.java
----------------------------------------------------------------------
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 0000000..f245a44
--- /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<T> extends BaseProcessor<T> {
+    private final Predicate<T> predicate;
+
+    public FilterProcessor(Predicate<T> predicate) {
+        this.predicate = predicate;
+    }
+
+    @Override
+    public void execute(T input) {
+        if (predicate.test(input)) {
+            context.forward(input);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/FlatMapProcessor.java
----------------------------------------------------------------------
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 0000000..fdc86fe
--- /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<T, R> extends BaseProcessor<T> {
+    private final FlatMapFunction<T, R> function;
+
+    public FlatMapProcessor(FlatMapFunction<T, R> function) {
+        this.function = function;
+    }
+
+    @Override
+    public void execute(T input) {
+        for (R res : function.apply(input)) {
+            context.forward(res);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/FlatMapValuesProcessor.java
----------------------------------------------------------------------
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 0000000..92953a3
--- /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<K, V, R> extends BaseProcessor<Pair<K, V>> {
+    private final FlatMapFunction<V, R> function;
+
+    public FlatMapValuesProcessor(FlatMapFunction<V, R> function) {
+        this.function = function;
+    }
+
+    @Override
+    protected void execute(Pair<K, V> input) {
+        for (R res : function.apply(input.getSecond())) {
+            context.forward(Pair.of(input.getFirst(), res));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/ForEachProcessor.java
----------------------------------------------------------------------
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 0000000..323b532
--- /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<T> extends BaseProcessor<T> {
+    private final Consumer<T> consumer;
+
+    public ForEachProcessor(Consumer<T> consumer) {
+        this.consumer = consumer;
+    }
+
+    @Override
+    public void execute(T input) {
+        consumer.accept(input);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/ForwardingProcessorContext.java
----------------------------------------------------------------------
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 0000000..49eb836
--- /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<String, ProcessorNode> streamToChildren;
+    private final Set<String> streams;
+
+    public ForwardingProcessorContext(ProcessorNode processorNode, Multimap<String, ProcessorNode> streamToChildren) {
+        this.processorNode = processorNode;
+        this.streamToChildren = streamToChildren;
+        this.streams = streamToChildren.keySet();
+    }
+
+    @Override
+    public <T> void forward(T input) {
+        if (PUNCTUATION.equals(input)) {
+            finishAllStreams();
+        } else {
+            executeAllStreams(input);
+        }
+    }
+
+    @Override
+    public <T> 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<String> getWindowedParentStreams() {
+        return processorNode.getWindowedParentStreams();
+    }
+
+    private void finishAllStreams() {
+        for (String stream : streams) {
+            finish(stream);
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private <T> void finish(String stream) {
+        for (ProcessorNode node : streamToChildren.get(stream)) {
+            LOG.debug("Punctuating processor: {}", node);
+            Processor<T> processor = (Processor<T>) node.getProcessor();
+            processor.punctuate(stream);
+        }
+    }
+
+    private <T> void executeAllStreams(T input) {
+        for (String stream : streams) {
+            execute(input, stream);
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private <T> void execute(T input, String stream) {
+        for (ProcessorNode node : streamToChildren.get(stream)) {
+            LOG.debug("Forward input: {} to processor node: {}", input, node);
+            Processor<T> processor = (Processor<T>) node.getProcessor();
+            processor.execute(input, stream);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/JoinProcessor.java
----------------------------------------------------------------------
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 0000000..d56cfea
--- /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<K, R, V1, V2> extends BaseProcessor<Pair<K, ?>> implements BatchProcessor {
+    private final ValueJoiner<V1, V2, R> valueJoiner;
+    private final String leftStream;
+    private final String rightStream;
+    private final List<Pair<K, V1>> leftRows = new ArrayList<>();
+    private final List<Pair<K, V2>> rightRows = new ArrayList<>();
+
+    public JoinProcessor(String leftStream, String rightStream, ValueJoiner<V1, V2, R> valueJoiner) {
+        this.valueJoiner = valueJoiner;
+        this.leftStream = leftStream;
+        this.rightStream = rightStream;
+    }
+
+    @Override
+    public void execute(Pair<K, ?> input, String sourceStream) {
+        K key = input.getFirst();
+        if (sourceStream.equals(leftStream)) {
+            V1 val = (V1) input.getSecond();
+            Pair<K, V1> 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<K, V2> 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<Pair<K, V1>> leftRows, List<Pair<K, V2>> rightRows) {
+        if (leftRows.size() <= rightRows.size()) {
+            for (Tuple3<K, V1, V2> res : join(getJoinTable(leftRows), rightRows)) {
+                context.forward(Pair.of(res._1, valueJoiner.apply(res._2, res._3)));
+            }
+        } else {
+            for (Tuple3<K, V2, V1> res : join(getJoinTable(rightRows), leftRows)) {
+                context.forward(Pair.of(res._1, valueJoiner.apply(res._3, res._2)));
+            }
+        }
+    }
+
+    private <T1, T2> List<Tuple3<K, T1, T2>> join(Multimap<K, T1> tab, List<Pair<K, T2>> rows) {
+        List<Tuple3<K, T1, T2>> res = new ArrayList<>();
+        for (Pair<K, T2> row : rows) {
+            for (T1 mapValue : tab.get(row.getFirst())) {
+                if (mapValue != null) {
+                    res.add(new Tuple3<>(row.getFirst(), mapValue, row.getSecond()));
+                }
+            }
+        }
+        return res;
+    }
+
+    private <T> Multimap<K, T> getJoinTable(List<Pair<K, T>> rows) {
+        Multimap<K, T> m = ArrayListMultimap.create();
+        for (Pair<K, T> v : rows) {
+            m.put(v.getFirst(), v.getSecond());
+        }
+        return m;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/MapProcessor.java
----------------------------------------------------------------------
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 0000000..e9f89a9
--- /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<T, R> extends BaseProcessor<T> {
+    private final Function<T, R> function;
+
+    public MapProcessor(Function<T, R> function) {
+        this.function = function;
+    }
+
+    @Override
+    public void execute(T input) {
+        context.forward(function.apply(input));
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/MapValuesProcessor.java
----------------------------------------------------------------------
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 0000000..a963e2e
--- /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<K, V, R> extends BaseProcessor<Pair<K, V>> {
+    private final Function<V, R> function;
+
+    public MapValuesProcessor(Function<V, R> function) {
+        this.function = function;
+    }
+
+    @Override
+    public void execute(Pair<K, V> input) {
+        context.forward(Pair.of(input.getFirst(), function.apply(input.getSecond())));
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/PeekProcessor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/PeekProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/PeekProcessor.java
new file mode 100644
index 0000000..3014a98
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/PeekProcessor.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.operations.Consumer;
+
+public class PeekProcessor<T> extends BaseProcessor<T> {
+    private final Consumer<T> consumer;
+
+    public PeekProcessor(Consumer<T> consumer) {
+        this.consumer = consumer;
+    }
+
+    @Override
+    public void execute(T input) {
+        consumer.accept(input);
+        context.forward(input);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java
new file mode 100644
index 0000000..5551682
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.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.processors;
+
+import java.io.Serializable;
+
+/**
+ * A processor processes a stream of elements and produces some result.
+ *
+ * @param <T> the type of the input that is processed
+ */
+public interface Processor<T> extends Serializable {
+    /**
+     * Initializes the processor. This is typically invoked from the underlying
+     * storm bolt's prepare method.
+     *
+     * @param context the processor context
+     */
+    void init(ProcessorContext context);
+
+    /**
+     * Executes some operation on the input and possibly emits some result.
+     *
+     * @param input    the input to be processed
+     * @param streamId the source stream id from where the input is received
+     */
+    void execute(T input, String streamId);
+
+    /**
+     * Punctuation marks end of a batch which can be used to compute and pass
+     * the results of one stage in the pipeline to the next. For e.g. emit the results of an aggregation.
+     *
+     * @param stream the stream id on which the punctuation arrived
+     */
+    void punctuate(String stream);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java
new file mode 100644
index 0000000..2a10a22
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java
@@ -0,0 +1,59 @@
+/**
+ * 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.io.Serializable;
+import java.util.Set;
+
+/**
+ * Context information passed to the {@link Processor}.
+ */
+public interface ProcessorContext extends Serializable {
+    /**
+     * Forwards the input to all downstream processors.
+     *
+     * @param input the input
+     * @param <T>   the type of the input
+     */
+    <T> void forward(T input);
+
+    /**
+     * Forwards the input to downstream processors at specified stream.
+     *
+     * @param input  the input
+     * @param stream the stream to forward
+     * @param <T>    the type of the input
+     */
+    <T> void forward(T input, String stream);
+
+    /**
+     * Returns if the processing is in a windowed context and should wait for
+     * punctuation before emitting results.
+     *
+     * @return if this is a windowed context or not
+     */
+    boolean isWindowed();
+
+    /**
+     * Returns the windowed parent streams. These are the streams
+     * where punctuations arrive.
+     *
+     * @return the windowed parent streams
+     */
+    Set<String> getWindowedParentStreams();
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceByKeyProcessor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceByKeyProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceByKeyProcessor.java
new file mode 100644
index 0000000..eba0de6
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceByKeyProcessor.java
@@ -0,0 +1,52 @@
+/**
+ * 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.Reducer;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class ReduceByKeyProcessor<K, V> extends BaseProcessor<Pair<K, V>> implements BatchProcessor {
+    private final Reducer<V> reducer;
+    private final Map<K, V> state = new HashMap<>();
+
+    public ReduceByKeyProcessor(Reducer<V> reducer) {
+        this.reducer = reducer;
+    }
+
+    @Override
+    public void execute(Pair<K, V> input) {
+        K key = input.getFirst();
+        V val = input.getSecond();
+        V agg = state.get(key);
+        agg = (agg == null) ? val : reducer.apply(agg, val);
+        state.put(key, agg);
+        mayBeForwardAggUpdate(Pair.of(key, agg));
+    }
+
+    @Override
+    public void finish() {
+        for (Map.Entry<K, V> entry : state.entrySet()) {
+            context.forward(Pair.of(entry.getKey(), entry.getValue()));
+        }
+        state.clear();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java
new file mode 100644
index 0000000..d64e114
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.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.Reducer;
+
+public class ReduceProcessor<T> extends BaseProcessor<T> implements BatchProcessor {
+    private final Reducer<T> reducer;
+    private T agg;
+
+    public ReduceProcessor(Reducer<T> reducer) {
+        this.reducer = reducer;
+    }
+
+    @Override
+    public void execute(T input) {
+        agg = (agg == null) ? input : reducer.apply(agg, input);
+        mayBeForwardAggUpdate(agg);
+    }
+
+    @Override
+    public void finish() {
+        context.forward(agg);
+        agg = null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/StateQueryProcessor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/StateQueryProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/StateQueryProcessor.java
new file mode 100644
index 0000000..e577a65
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/StateQueryProcessor.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.processors;
+
+import org.apache.storm.state.KeyValueState;
+import org.apache.storm.streams.Pair;
+import org.apache.storm.streams.StreamState;
+
+public class StateQueryProcessor<K, V> extends BaseProcessor<K> implements StatefulProcessor<K, V> {
+    private final StreamState<K, V> streamState;
+    private KeyValueState<K, V> keyValueState;
+
+    public StateQueryProcessor(StreamState<K, V> streamState) {
+        this.streamState = streamState;
+    }
+
+    @Override
+    public void initState(KeyValueState<K, V> keyValueState) {
+        this.keyValueState = keyValueState;
+    }
+
+    public StreamState<K, V> getStreamState() {
+        return streamState;
+    }
+
+    @Override
+    protected void execute(K key) {
+        V value = keyValueState.get(key);
+        if (value != null) {
+            context.forward(Pair.of(key, value));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/StatefulProcessor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/StatefulProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/StatefulProcessor.java
new file mode 100644
index 0000000..acc4188
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/StatefulProcessor.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.state.KeyValueState;
+
+/**
+ * Top level interface for processors that does stateful processing.
+ *
+ * @param <K> the key type
+ * @param <V> the value type
+ */
+public interface StatefulProcessor<K, V> {
+    /**
+     * Initialize the state of the processor with the given
+     * {@link KeyValueState}
+     *
+     * @param keyValueState the key-value state
+     */
+    void initState(KeyValueState<K, V> keyValueState);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/processors/UpdateStateByKeyProcessor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/UpdateStateByKeyProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/UpdateStateByKeyProcessor.java
new file mode 100644
index 0000000..9e068a0
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/UpdateStateByKeyProcessor.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.processors;
+
+import org.apache.storm.state.KeyValueState;
+import org.apache.storm.streams.Pair;
+import org.apache.storm.streams.operations.Aggregator;
+
+public class UpdateStateByKeyProcessor<K, V, R> extends BaseProcessor<Pair<K, V>> implements StatefulProcessor<K, R> {
+    private final Aggregator<V, R> aggregator;
+    private KeyValueState<K, R> keyValueState;
+
+    public UpdateStateByKeyProcessor(Aggregator<V, R> aggregator) {
+        this.aggregator = aggregator;
+    }
+
+    @Override
+    public void initState(KeyValueState<K, R> keyValueState) {
+        this.keyValueState = keyValueState;
+    }
+
+    @Override
+    protected void execute(Pair<K, V> input) {
+        K key = input.getFirst();
+        V val = input.getSecond();
+        R agg = keyValueState.get(key);
+        if (agg == null) {
+            agg = aggregator.init();
+        }
+        R newAgg = aggregator.apply(val, agg);
+        keyValueState.put(key, newAgg);
+        context.forward(Pair.of(key, newAgg));
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/windowing/BaseWindow.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/windowing/BaseWindow.java b/storm-core/src/jvm/org/apache/storm/streams/windowing/BaseWindow.java
new file mode 100644
index 0000000..4526fa5
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/windowing/BaseWindow.java
@@ -0,0 +1,64 @@
+/**
+ * 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.windowing;
+
+import static org.apache.storm.topology.base.BaseWindowedBolt.Duration;
+
+public abstract class BaseWindow<L, I> implements Window<L, I> {
+    protected String timestampField;
+    protected String lateTupleStream;
+    protected Duration lag;
+
+    @Override
+    public String getTimestampField() {
+        return timestampField;
+    }
+
+    @Override
+    public String getLateTupleStream() {
+        return lateTupleStream;
+    }
+
+    @Override
+    public Duration getLag() {
+        return lag;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        BaseWindow<?, ?> that = (BaseWindow<?, ?>) o;
+
+        if (timestampField != null ? !timestampField.equals(that.timestampField) : that.timestampField != null)
+            return false;
+        if (lateTupleStream != null ? !lateTupleStream.equals(that.lateTupleStream) : that.lateTupleStream != null)
+            return false;
+        return lag != null ? lag.equals(that.lag) : that.lag == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = timestampField != null ? timestampField.hashCode() : 0;
+        result = 31 * result + (lateTupleStream != null ? lateTupleStream.hashCode() : 0);
+        result = 31 * result + (lag != null ? lag.hashCode() : 0);
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/windowing/SlidingWindows.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/windowing/SlidingWindows.java b/storm-core/src/jvm/org/apache/storm/streams/windowing/SlidingWindows.java
new file mode 100644
index 0000000..275d3c3
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/windowing/SlidingWindows.java
@@ -0,0 +1,151 @@
+/**
+ * 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.windowing;
+
+import org.apache.storm.topology.base.BaseWindowedBolt;
+import org.apache.storm.topology.base.BaseWindowedBolt.Count;
+import org.apache.storm.topology.base.BaseWindowedBolt.Duration;
+
+/**
+ * A sliding window specification based on a window length and sliding interval.
+ *
+ * @param <L> The type of the window length (e.g. Duration or Count)
+ * @param <I> The type of the sliding interval (e.g. Duration or Count)
+ */
+public class SlidingWindows<L, I> extends BaseWindow<L, I> {
+    private final L windowLength;
+    private final I slidingInterval;
+
+    private SlidingWindows(L windowLength, I slidingInterval) {
+        this.windowLength = windowLength;
+        this.slidingInterval = slidingInterval;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public L getWindowLength() {
+        return windowLength;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public I getSlidingInterval() {
+        return slidingInterval;
+    }
+
+    /**
+     * Count based sliding window configuration.
+     *
+     * @param windowLength    the number of tuples in the window
+     * @param slidingInterval the number of tuples after which the window slides
+     */
+    public static SlidingWindows<Count, Count> of(Count windowLength, Count slidingInterval) {
+        return new SlidingWindows<>(windowLength, slidingInterval);
+    }
+
+    /**
+     * Time duration based sliding window configuration.
+     *
+     * @param windowLength    the time duration of the window
+     * @param slidingInterval the time duration after which the window slides
+     */
+    public static SlidingWindows<Duration, Duration> of(Duration windowLength, Duration slidingInterval) {
+        return new SlidingWindows<>(windowLength, slidingInterval);
+    }
+
+    /**
+     * Tuple count and time duration based sliding window configuration.
+     *
+     * @param windowLength    the number of tuples in the window
+     * @param slidingInterval the time duration after which the window slides
+     */
+    public static SlidingWindows<Count, Duration> of(Count windowLength, Duration slidingInterval) {
+        return new SlidingWindows<>(windowLength, slidingInterval);
+    }
+
+    /**
+     * Time duration and count based sliding window configuration.
+     *
+     * @param windowLength    the time duration of the window
+     * @param slidingInterval the number of tuples after which the window slides
+     */
+    public static SlidingWindows<Duration, Count> of(Duration windowLength, Count slidingInterval) {
+        return new SlidingWindows<>(windowLength, slidingInterval);
+    }
+
+    /**
+     * The name of the field in the tuple that contains the timestamp when the event occurred as a long value.
+     * This is used of event-time based processing. If this config is set and the field is not present in the incoming tuple,
+     * an {@link IllegalArgumentException} will be thrown.
+     *
+     * @param fieldName the name of the field that contains the timestamp
+     */
+    public SlidingWindows<L, I> withTimestampField(String fieldName) {
+        timestampField = fieldName;
+        return this;
+    }
+
+    /**
+     * Specify a stream id on which late tuples are going to be emitted. They are going to be accessible via the
+     * {@link org.apache.storm.topology.WindowedBoltExecutor#LATE_TUPLE_FIELD} field.
+     * It must be defined on a per-component basis, and in conjunction with the
+     * {@link BaseWindowedBolt#withTimestampField}, otherwise {@link IllegalArgumentException} will be thrown.
+     *
+     * @param streamId the name of the stream used to emit late tuples on
+     */
+    public SlidingWindows<L, I> withLateTupleStream(String streamId) {
+        lateTupleStream = streamId;
+        return this;
+    }
+
+    /**
+     * Specify the maximum time lag of the tuple timestamp in milliseconds. It means that the tuple timestamps
+     * cannot be out of order by more than this amount.
+     *
+     * @param duration the max lag duration
+     */
+    public SlidingWindows<L, I> withLag(Duration duration) {
+        lag = duration;
+        return this;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        if (!super.equals(o)) return false;
+
+        SlidingWindows<?, ?> that = (SlidingWindows<?, ?>) o;
+
+        if (windowLength != null ? !windowLength.equals(that.windowLength) : that.windowLength != null) return false;
+        return slidingInterval != null ? slidingInterval.equals(that.slidingInterval) : that.slidingInterval == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = super.hashCode();
+        result = 31 * result + (windowLength != null ? windowLength.hashCode() : 0);
+        result = 31 * result + (slidingInterval != null ? slidingInterval.hashCode() : 0);
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/windowing/TumblingWindows.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/windowing/TumblingWindows.java b/storm-core/src/jvm/org/apache/storm/streams/windowing/TumblingWindows.java
new file mode 100644
index 0000000..71ed617
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/windowing/TumblingWindows.java
@@ -0,0 +1,119 @@
+/**
+ * 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.windowing;
+
+import org.apache.storm.topology.base.BaseWindowedBolt;
+
+import static org.apache.storm.topology.base.BaseWindowedBolt.Count;
+import static org.apache.storm.topology.base.BaseWindowedBolt.Duration;
+
+/**
+ * A tumbling window specification. The window tumbles after the specified window length.
+ *
+ * @param <L> the type of the length of the window (e.g Count, Duration)
+ */
+public class TumblingWindows<L> extends BaseWindow<L, L> {
+    private final L windowLength;
+
+    private TumblingWindows(L windowLength) {
+        this.windowLength = windowLength;
+    }
+
+    @Override
+    public L getWindowLength() {
+        return windowLength;
+    }
+
+    @Override
+    public L getSlidingInterval() {
+        return windowLength;
+    }
+
+    /**
+     * A count based tumbling window.
+     *
+     * @param count the number of tuples after which the window tumbles
+     */
+    public static TumblingWindows<Count> of(Count count) {
+        return new TumblingWindows<>(count);
+    }
+
+    /**
+     * A time duration based tumbling window.
+     *
+     * @param duration the time duration after which the window tumbles
+     */
+    public static TumblingWindows<Duration> of(Duration duration) {
+        return new TumblingWindows<>(duration);
+    }
+
+    /**
+     * The name of the field in the tuple that contains the timestamp when the event occurred as a long value.
+     * This is used of event-time based processing. If this config is set and the field is not present in the incoming tuple,
+     * an {@link IllegalArgumentException} will be thrown.
+     *
+     * @param fieldName the name of the field that contains the timestamp
+     */
+    public TumblingWindows<L> withTimestampField(String fieldName) {
+        timestampField = fieldName;
+        return this;
+    }
+
+    /**
+     * Specify a stream id on which late tuples are going to be emitted. They are going to be accessible via the
+     * {@link org.apache.storm.topology.WindowedBoltExecutor#LATE_TUPLE_FIELD} field.
+     * It must be defined on a per-component basis, and in conjunction with the
+     * {@link BaseWindowedBolt#withTimestampField}, otherwise {@link IllegalArgumentException} will be thrown.
+     *
+     * @param streamId the name of the stream used to emit late tuples on
+     */
+    public TumblingWindows<L> withLateTupleStream(String streamId) {
+        lateTupleStream = streamId;
+        return this;
+    }
+
+    /**
+     * Specify the maximum time lag of the tuple timestamp in milliseconds. It means that the tuple timestamps
+     * cannot be out of order by more than this amount.
+     *
+     * @param duration the max lag duration
+     */
+    public TumblingWindows<L> withLag(Duration duration) {
+        lag = duration;
+        return this;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        if (!super.equals(o)) return false;
+
+        TumblingWindows<?> that = (TumblingWindows<?>) o;
+
+        return windowLength != null ? windowLength.equals(that.windowLength) : that.windowLength == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = super.hashCode();
+        result = 31 * result + (windowLength != null ? windowLength.hashCode() : 0);
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/windowing/Window.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/windowing/Window.java b/storm-core/src/jvm/org/apache/storm/streams/windowing/Window.java
new file mode 100644
index 0000000..40378e3
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/windowing/Window.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.windowing;
+
+import java.io.Serializable;
+
+import static org.apache.storm.topology.base.BaseWindowedBolt.Duration;
+
+/**
+ * The window specification within {@link org.apache.storm.streams.Stream}.
+ *
+ * @param <L> the type of window length parameter (E.g. Count, Duration)
+ * @param <I> the type of the sliding interval parameter (E.g. Count, Duration)
+ */
+public interface Window<L, I> extends Serializable {
+
+    /**
+     * The length of the window.
+     *
+     * @return the window length
+     */
+    L getWindowLength();
+
+    /**
+     * The sliding interval of the window.
+     *
+     * @return the sliding interval
+     */
+    I getSlidingInterval();
+
+    /**
+     * The name of the field in the tuple that contains the timestamp when the event occurred as a long value.
+     * This is used of event-time based processing. If this config is set and the field is not present in the incoming tuple,
+     * an {@link IllegalArgumentException} will be thrown.
+     *
+     * @return the timestamp field.
+     */
+    String getTimestampField();
+
+    /**
+     * The name of the stream where late arriving tuples should be emitted. If this is not provided,
+     * the late tuples would be discarded.
+     *
+     * @return the name of the stream used to emit late tuples on
+     */
+    String getLateTupleStream();
+
+    /**
+     * The maximum time lag of the tuple timestamp in milliseconds. It means that the tuple timestamps
+     * cannot be out of order by more than this amount.
+     *
+     * @return the lag
+     */
+    Duration getLag();
+}


[11/12] storm git commit: Merge branch 'streams' of https://github.com/arunmahadevan/storm into STORM-1961-merge

Posted by ka...@apache.org.
Merge branch 'streams' of https://github.com/arunmahadevan/storm into STORM-1961-merge


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/00b0fd75
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/00b0fd75
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/00b0fd75

Branch: refs/heads/master
Commit: 00b0fd75025b87c96fd535387964f16be42cb776
Parents: a7cdfef 37ee12c
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Tue Feb 7 10:23:16 2017 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Tue Feb 7 10:23:16 2017 +0900

----------------------------------------------------------------------
 .../storm/starter/streams/AggregateExample.java |  95 +++
 .../storm/starter/streams/BranchExample.java    |  71 +++
 .../streams/GroupByKeyAndWindowExample.java     | 114 ++++
 .../storm/starter/streams/JoinExample.java      | 109 ++++
 .../starter/streams/StateQueryExample.java      | 116 ++++
 .../starter/streams/StatefulWordCount.java      |  88 +++
 .../starter/streams/TypedTupleExample.java      |  63 ++
 .../starter/streams/WindowedWordCount.java      |  79 +++
 .../storm/starter/streams/WordCountToBolt.java  | 105 ++++
 .../storm/annotation/InterfaceStability.java    |  54 ++
 .../src/jvm/org/apache/storm/streams/Edge.java  |  41 ++
 .../org/apache/storm/streams/GroupingInfo.java  | 100 +++
 .../src/jvm/org/apache/storm/streams/Node.java  | 170 ++++++
 .../src/jvm/org/apache/storm/streams/Pair.java  | 103 ++++
 .../org/apache/storm/streams/PairStream.java    | 525 ++++++++++++++++
 .../org/apache/storm/streams/PartitionNode.java |  37 ++
 .../org/apache/storm/streams/ProcessorBolt.java |  70 +++
 .../storm/streams/ProcessorBoltDelegate.java    | 348 +++++++++++
 .../org/apache/storm/streams/ProcessorNode.java |  83 +++
 .../apache/storm/streams/RefCountedTuple.java   |  64 ++
 .../jvm/org/apache/storm/streams/SinkNode.java  |  44 ++
 .../jvm/org/apache/storm/streams/SpoutNode.java |  48 ++
 .../storm/streams/StatefulProcessorBolt.java    | 116 ++++
 .../jvm/org/apache/storm/streams/Stream.java    | 515 ++++++++++++++++
 .../org/apache/storm/streams/StreamBolt.java    |  38 ++
 .../org/apache/storm/streams/StreamBuilder.java | 602 +++++++++++++++++++
 .../org/apache/storm/streams/StreamState.java   |  43 ++
 .../org/apache/storm/streams/StreamUtil.java    |  69 +++
 .../storm/streams/StreamsEdgeFactory.java       |  29 +
 .../org/apache/storm/streams/UniqueIdGen.java   |  56 ++
 .../org/apache/storm/streams/WindowNode.java    |  39 ++
 .../storm/streams/WindowedProcessorBolt.java    | 137 +++++
 .../storm/streams/operations/BiFunction.java    |  37 ++
 .../streams/operations/CombinerAggregator.java  |  97 +++
 .../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 +
 .../streams/operations/PairFlatMapFunction.java |  30 +
 .../storm/streams/operations/PairFunction.java  |  30 +
 .../streams/operations/PairValueJoiner.java     |  40 ++
 .../storm/streams/operations/Predicate.java     |  33 +
 .../storm/streams/operations/PrintConsumer.java |  30 +
 .../storm/streams/operations/Reducer.java       |  35 ++
 .../storm/streams/operations/StateUpdater.java  |  67 +++
 .../storm/streams/operations/ValueJoiner.java   |  36 ++
 .../streams/operations/aggregators/Count.java   |  47 ++
 .../streams/operations/aggregators/LongSum.java |  45 ++
 .../operations/mappers/PairValueMapper.java     |  51 ++
 .../operations/mappers/TupleValueMapper.java    |  30 +
 .../operations/mappers/TupleValueMappers.java   | 174 ++++++
 .../streams/operations/mappers/ValueMapper.java |  45 ++
 .../operations/mappers/ValuesMapper.java        |  48 ++
 .../processors/AggregateByKeyProcessor.java     |  78 +++
 .../streams/processors/AggregateProcessor.java  |  69 +++
 .../storm/streams/processors/BaseProcessor.java | 109 ++++
 .../streams/processors/BatchProcessor.java      |  25 +
 .../streams/processors/BranchProcessor.java     |  41 ++
 .../processors/ChainedProcessorContext.java     |  66 ++
 .../processors/EmittingProcessorContext.java    | 155 +++++
 .../streams/processors/FilterProcessor.java     |  35 ++
 .../streams/processors/FlatMapProcessor.java    |  35 ++
 .../processors/FlatMapValuesProcessor.java      |  36 ++
 .../streams/processors/ForEachProcessor.java    |  33 +
 .../processors/ForwardingProcessorContext.java  | 102 ++++
 .../storm/streams/processors/JoinProcessor.java | 152 +++++
 .../storm/streams/processors/MapProcessor.java  |  33 +
 .../streams/processors/MapValuesProcessor.java  |  34 ++
 .../MergeAggregateByKeyProcessor.java           |  54 ++
 .../processors/MergeAggregateProcessor.java     |  47 ++
 .../storm/streams/processors/PeekProcessor.java |  34 ++
 .../storm/streams/processors/Processor.java     |  54 ++
 .../streams/processors/ProcessorContext.java    |  62 ++
 .../processors/ReduceByKeyProcessor.java        |  52 ++
 .../streams/processors/ReduceProcessor.java     |  43 ++
 .../streams/processors/StateQueryProcessor.java |  48 ++
 .../streams/processors/StatefulProcessor.java   |  36 ++
 .../processors/UpdateStateByKeyProcessor.java   |  49 ++
 .../org/apache/storm/streams/tuple/Tuple10.java | 112 ++++
 .../org/apache/storm/streams/tuple/Tuple3.java  |  70 +++
 .../org/apache/storm/streams/tuple/Tuple4.java  |  76 +++
 .../org/apache/storm/streams/tuple/Tuple5.java  |  82 +++
 .../org/apache/storm/streams/tuple/Tuple6.java  |  89 +++
 .../org/apache/storm/streams/tuple/Tuple7.java  |  94 +++
 .../org/apache/storm/streams/tuple/Tuple8.java  | 100 +++
 .../org/apache/storm/streams/tuple/Tuple9.java  | 106 ++++
 .../storm/streams/windowing/BaseWindow.java     |  64 ++
 .../storm/streams/windowing/SlidingWindows.java | 151 +++++
 .../streams/windowing/TumblingWindows.java      | 119 ++++
 .../apache/storm/streams/windowing/Window.java  |  70 +++
 .../topology/StatefulWindowedBoltExecutor.java  |   4 +-
 .../storm/topology/WindowedBoltExecutor.java    |   4 +-
 .../storm/topology/base/BaseWindowedBolt.java   |  37 +-
 .../windowing/AbstractTridentWindowManager.java |   2 +-
 .../storm/windowing/CountEvictionPolicy.java    |   8 +-
 .../apache/storm/windowing/EvictionPolicy.java  |   7 +
 .../storm/windowing/TimeEvictionPolicy.java     |  17 +-
 .../apache/storm/windowing/TupleWindowImpl.java |  11 +
 .../windowing/WatermarkCountEvictionPolicy.java |  10 +-
 .../windowing/WatermarkTimeEvictionPolicy.java  |   4 +-
 .../jvm/org/apache/storm/windowing/Window.java  |   7 +
 .../windowing/WindowLifecycleListener.java      |   3 +-
 .../apache/storm/windowing/WindowManager.java   |   2 +-
 .../apache/storm/streams/ProcessorBoltTest.java | 174 ++++++
 .../streams/StatefulProcessorBoltTest.java      | 111 ++++
 .../apache/storm/streams/StreamBuilderTest.java | 268 +++++++++
 .../streams/WindowedProcessorBoltTest.java      | 110 ++++
 .../streams/processors/JoinProcessorTest.java   | 108 ++++
 .../storm/windowing/WindowManagerTest.java      |   2 +-
 110 files changed, 8671 insertions(+), 29 deletions(-)
----------------------------------------------------------------------



[04/12] storm git commit: [STORM-1961] Stream api for storm core use cases

Posted by ka...@apache.org.
[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.


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/e251573d
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/e251573d
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/e251573d

Branch: refs/heads/master
Commit: e251573d524f6a6a11c7821ba93fd155d9beb770
Parents: e3b2f96
Author: Arun Mahadevan <ar...@apache.org>
Authored: Mon Sep 19 23:50:16 2016 +0530
Committer: Arun Mahadevan <ar...@apache.org>
Committed: Fri Jan 13 01:20:43 2017 +0530

----------------------------------------------------------------------
 .../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 +++
 .../storm/starter/streams/WordCountToBolt.java  | 111 ++++
 .../src/jvm/org/apache/storm/streams/Edge.java  |  41 ++
 .../org/apache/storm/streams/GroupingInfo.java  |  84 +++
 .../src/jvm/org/apache/storm/streams/Node.java  | 129 ++++
 .../src/jvm/org/apache/storm/streams/Pair.java  |  78 +++
 .../org/apache/storm/streams/PairStream.java    | 241 ++++++++
 .../org/apache/storm/streams/PartitionNode.java |  42 ++
 .../org/apache/storm/streams/ProcessorBolt.java |  70 +++
 .../storm/streams/ProcessorBoltDelegate.java    | 285 +++++++++
 .../org/apache/storm/streams/ProcessorNode.java |  81 +++
 .../apache/storm/streams/RefCountedTuple.java   |  64 ++
 .../jvm/org/apache/storm/streams/SinkNode.java  |  44 ++
 .../jvm/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 ++
 .../org/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 +
 .../streams/operations/PairFlatMapFunction.java |  30 +
 .../storm/streams/operations/PairFunction.java  |  30 +
 .../streams/operations/PairValueJoiner.java     |  40 ++
 .../storm/streams/operations/Predicate.java     |  33 ++
 .../storm/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 +
 .../streams/operations/mappers/ValueMapper.java |  45 ++
 .../operations/mappers/ValuesMapper.java        |  48 ++
 .../processors/AggregateByKeyProcessor.java     |  54 ++
 .../streams/processors/AggregateProcessor.java  |  45 ++
 .../storm/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 ++
 .../processors/ForwardingProcessorContext.java  | 102 ++++
 .../storm/streams/processors/JoinProcessor.java | 112 ++++
 .../storm/streams/processors/MapProcessor.java  |  33 ++
 .../streams/processors/MapValuesProcessor.java  |  34 ++
 .../storm/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 ++
 .../streams/processors/StateQueryProcessor.java |  48 ++
 .../streams/processors/StatefulProcessor.java   |  36 ++
 .../processors/UpdateStateByKeyProcessor.java   |  49 ++
 .../storm/streams/windowing/BaseWindow.java     |  64 ++
 .../storm/streams/windowing/SlidingWindows.java | 151 +++++
 .../streams/windowing/TumblingWindows.java      | 119 ++++
 .../apache/storm/streams/windowing/Window.java  |  70 +++
 .../topology/StatefulWindowedBoltExecutor.java  |   4 +-
 .../storm/topology/WindowedBoltExecutor.java    |   4 +-
 .../storm/topology/base/BaseWindowedBolt.java   |  37 +-
 .../windowing/AbstractTridentWindowManager.java |   2 +-
 .../storm/windowing/CountEvictionPolicy.java    |   8 +-
 .../apache/storm/windowing/EvictionPolicy.java  |   7 +
 .../storm/windowing/TimeEvictionPolicy.java     |  17 +-
 .../apache/storm/windowing/TupleWindowImpl.java |  11 +
 .../windowing/WatermarkCountEvictionPolicy.java |  10 +-
 .../windowing/WatermarkTimeEvictionPolicy.java  |   4 +-
 .../jvm/org/apache/storm/windowing/Window.java  |   7 +
 .../windowing/WindowLifecycleListener.java      |   3 +-
 .../apache/storm/windowing/WindowManager.java   |   2 +-
 .../apache/storm/streams/ProcessorBoltTest.java | 165 ++++++
 .../streams/StatefulProcessorBoltTest.java      | 100 ++++
 .../apache/storm/streams/StreamBuilderTest.java | 219 +++++++
 .../streams/WindowedProcessorBoltTest.java      | 110 ++++
 .../storm/windowing/WindowManagerTest.java      |   2 +-
 94 files changed, 6505 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/BranchExample.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 0000000..f5400a5
--- /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<Integer>[] evenAndOdd = builder
+                /*
+                 * Create a stream of random numbers from a spout that
+                 * emits random integers by extracting the tuple value at index 0.
+                 */
+                .newStream(new RandomIntegerSpout(), new ValueMapper<Integer>(0))
+                /*
+                 * Split the stream of numbers into streams of
+                 * even and odd numbers. The first stream contains even
+                 * and the second contains odd numbers.
+                 */
+                .branch(x -> (x % 2) == 0,
+                        x -> (x % 2) == 1);
+
+        evenAndOdd[0].forEach(x -> LOG.info("EVEN> " + x));
+        evenAndOdd[1].forEach(x -> LOG.info("ODD > " + x));
+
+        Config config = new Config();
+        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();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/GroupByKeyAndWindowExample.java
----------------------------------------------------------------------
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 0000000..6b505bd
--- /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<String, Double>(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<String, Double>(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<List<Values>> 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"));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/JoinExample.java
----------------------------------------------------------------------
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 0000000..0b15615
--- /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<Integer, Integer> squares = builder
+                .newStream(new NumberSpout(x -> x * x),
+                        new PairValueMapper<>(0, 1));
+        // a stream of (number, cube) pairs
+        PairStream<Integer, Integer> cubes = builder
+                .newStream(new NumberSpout(x -> x * x * x),
+                        new PairValueMapper<>(0, 1));
+
+        // create a windowed stream of five seconds duration
+        squares.window(TumblingWindows.of(Duration.seconds(5)))
+                /*
+                 * Join the squares and the cubes stream within the window.
+                 * The values in the squares stream having the same key as that
+                 * of the cubes stream within the window will be joined together.
+                 */
+                .join(cubes)
+                /**
+                 * The results should be of the form (number, (square, cube))
+                 */
+                .print();
+
+        Config config = new Config();
+        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<Integer, Integer> function;
+        private SpoutOutputCollector collector;
+        private int i = 1;
+
+        NumberSpout(Function<Integer, Integer> 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"));
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java
----------------------------------------------------------------------
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 0000000..e76dd3c
--- /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
+ * <p>
+ * 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.
+ * <p/>
+ * <pre>
+ * topology.state.provider.config: '{"keyClass":"...", "valueClass":"...",
+ *  "keySerializerClass":"...", "valueSerializerClass":"...",
+ *  "jedisPoolConfig":{"host":"localhost", "port":6379,
+ *  "timeout":2000, "database":0, "password":"xyz"}}'
+ * </pre>
+ */
+public class StateQueryExample {
+    public static void main(String[] args) throws Exception {
+        StreamBuilder builder = new StreamBuilder();
+        StreamState<String, Long> ss = builder.newStream(new TestWordSpout(), new ValueMapper<String>(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<String>(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"));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java
----------------------------------------------------------------------
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 0000000..f6ae6b0
--- /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.
+ * <p>
+ * 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.
+ * <p/>
+ * <pre>
+ * topology.state.provider.config: '{"keyClass":"...", "valueClass":"...",
+ *  "keySerializerClass":"...", "valueSerializerClass":"...",
+ *  "jedisPoolConfig":{"host":"localhost", "port":6379,
+ *  "timeout":2000, "database":0, "password":"xyz"}}'
+ * </pre>
+ */
+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<String>(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();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java
----------------------------------------------------------------------
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 0000000..c6e2f4a
--- /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<String>(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();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java
----------------------------------------------------------------------
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 0000000..a711696
--- /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<String>(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"));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/Edge.java
----------------------------------------------------------------------
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 0000000..9b13562
--- /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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/GroupingInfo.java
----------------------------------------------------------------------
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 0000000..81def4b
--- /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 +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/Node.java
----------------------------------------------------------------------
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 0000000..f9de390
--- /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<String> outputStreams;
+    protected final Fields outputFields;
+    protected String componentId;
+    protected int parallelism;
+    // the parent streams that this node subscribes to
+    private final Multimap<Node, String> parentStreams = ArrayListMultimap.create();
+
+    Node(Set<String> 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<String> getOutputStreams() {
+        return Collections.unmodifiableSet(outputStreams);
+    }
+
+    Collection<String> getParentStreams(Node parent) {
+        return parentStreams.get(parent);
+    }
+
+    Set<Node> getParents(String stream) {
+        Multimap<String, Node> rev = Multimaps.invertFrom(parentStreams, ArrayListMultimap.<String, Node>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<String, StreamInfo> 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 +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/Pair.java
----------------------------------------------------------------------
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 0000000..0044359
--- /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 <T1> the type of the first value
+ * @param <T2> the type of the second value
+ */
+public final class Pair<T1, T2> 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 <T1>   the type of the first value
+     * @param <T2>   the type of the second value
+     * @return a new pair of values
+     */
+    public static <T1, T2> Pair<T1, T2> of(T1 first, T2 second) {
+        return new Pair<>(first, second);
+    }
+
+    @Override
+    public String toString() {
+        return "(" + first + ", " + second + ')';
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/PairStream.java
----------------------------------------------------------------------
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 0000000..2d18b30
--- /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 <K> the key type
+ * @param <V> the value type
+ */
+public class PairStream<K, V> extends Stream<Pair<K, V>> {
+
+    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 <R>      the result type
+     * @return the new stream
+     */
+    public <R> PairStream<K, R> mapValues(Function<? super V, ? extends R> function) {
+        return new PairStream<>(streamBuilder, addProcessorNode(new MapValuesProcessor<>(function), KEY_VALUE));
+    }
+
+    /**
+     * 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 <R>      the result type
+     * @return the new stream
+     */
+    public <R> PairStream<K, R> flatMapValues(FlatMapFunction<V, R> 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 <R>        the result type
+     * @return the new stream
+     */
+    public <R> PairStream<K, R> 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<K, V> reduceByKey(Reducer<V> 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<K, V> 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<K, Iterable<V>> 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<K, V> reduceByKeyAndWindow(Reducer<V> reducer, Window<?, ?> window) {
+        return groupByKey().window(window).reduceByKey(reducer);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public PairStream<K, V> peek(Consumer<? super Pair<K, V>> action) {
+        return toPairStream(super.peek(action));
+    }
+
+    /**
+     * Join the values of this stream with the values having the same key from the other stream.
+     * <p>
+     * Note: The parallelism and windowing parameters (if windowed) of this stream is carried forward to the joined stream.
+     * </p>
+     *
+     * @param otherStream the other stream
+     * @param <V1>        the type of the values in the other stream
+     * @return the new stream
+     */
+    public <V1> PairStream<K, Pair<V, V1>> join(PairStream<K, V1> otherStream) {
+        return join(otherStream, new PairValueJoiner<>());
+    }
+
+    /**
+     * Join the values of this stream with the values having the same key from the other stream.
+     * <p>
+     * Note: The parallelism and windowing parameters (if windowed) of this stream is carried forward to the joined stream.
+     * </p>
+     *
+     * @param otherStream the other stream
+     * @param valueJoiner the {@link ValueJoiner}
+     * @param <R>         the type of the values resulting from the join
+     * @param <V1>        the type of the values in the other stream
+     * @return the new stream
+     */
+    public <R, V1> PairStream<K, R> join(PairStream<K, V1> 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<K, V> window(Window<?, ?> window) {
+        return toPairStream(super.window(window));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public PairStream<K, V> repartition(int parallelism) {
+        return toPairStream(super.repartition(parallelism));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    @SuppressWarnings("unchecked")
+    public PairStream<K, V>[] branch(Predicate<Pair<K, V>>... predicates) {
+        List<PairStream<K, V>> pairStreams = new ArrayList<>();
+        for (Stream<Pair<K, V>> 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 <R>        the result type
+     * @return the {@link StreamState} which can be used to query the state
+     */
+    public <R> StreamState<K, R> updateStateByKey(Aggregator<? super V, ? extends R> aggregator) {
+        return new StreamState<>(
+                new PairStream<>(streamBuilder, addProcessorNode(new UpdateStateByKeyProcessor<>(aggregator), KEY_VALUE)));
+    }
+
+    private PairStream<K, V> partitionBy(Fields fields) {
+        return new PairStream<>(
+                streamBuilder,
+                addNode(new PartitionNode(stream, node.getOutputFields(), GroupingInfo.fields(fields))));
+    }
+
+    private PairStream<K, V> toPairStream(Stream<Pair<K, V>> stream) {
+        return new PairStream<>(stream.streamBuilder, stream.node);
+    }
+
+    // used internally to merge values in groupByKeyAndWindow
+    private static class MergeValues<V> implements Aggregator<V, ArrayList<V>> {
+        @Override
+        public ArrayList<V> init() {
+            return new ArrayList<>();
+        }
+
+        @Override
+        public ArrayList<V> apply(V value, ArrayList<V> aggregate) {
+            aggregate.add(value);
+            return aggregate;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/PartitionNode.java
----------------------------------------------------------------------
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 0000000..ca92def
--- /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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/ProcessorBolt.java
----------------------------------------------------------------------
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 0000000..f1163ca
--- /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<Node, Edge> graph, List<ProcessorNode> 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<String, ProcessorNode> streamToInitialProcessors) {
+        delegate.setStreamToInitialProcessors(streamToInitialProcessors);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java
----------------------------------------------------------------------
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 0000000..5bc6fff
--- /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<Node, Edge> graph;
+    private final List<ProcessorNode> nodes;
+    private Map stormConf;
+    private TopologyContext topologyContext;
+    private OutputCollector outputCollector;
+    private final List<ProcessorNode> outgoingProcessors = new ArrayList<>();
+    private final Set<EmittingProcessorContext> emittingProcessorContexts = new HashSet<>();
+    private final Map<ProcessorNode, Set<String>> punctuationState = new HashMap<>();
+    private Multimap<String, ProcessorNode> streamToInitialProcessors;
+    private String timestampField;
+
+    ProcessorBoltDelegate(String id, DirectedGraph<Node, Edge> graph, List<ProcessorNode> nodes) {
+        this.id = id;
+        this.graph = graph;
+        this.nodes = new ArrayList<>(nodes);
+    }
+
+    String getId() {
+        return id;
+    }
+
+    void addNodes(Collection<ProcessorNode> nodes) {
+        this.nodes.addAll(nodes);
+    }
+
+    List<ProcessorNode> getNodes() {
+        return Collections.unmodifiableList(nodes);
+    }
+
+    void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        this.stormConf = stormConf;
+        topologyContext = context;
+        outputCollector = collector;
+        DirectedSubgraph<Node, Edge> subgraph = new DirectedSubgraph<>(graph, new HashSet<>(nodes), null);
+        TopologicalOrderIterator<Node, Edge> 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<ProcessorNode> children = StreamUtil.getChildren(subgraph, processorNode);
+            ProcessorContext processorContext;
+            if (children.isEmpty()) {
+                processorContext = createEmittingContext(processorNode);
+            } else {
+                Multimap<String, ProcessorNode> 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<String> 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<Object, String> getValueAndStream(Tuple input) {
+        Object value;
+        String stream;
+        // if tuple arrives from a spout, it can be passed as is
+        // otherwise the value is in the first field of the tuple
+        if (input.getSourceComponent().startsWith("spout")) {
+            value = input;
+            stream = input.getSourceGlobalStreamId().get_componentId() + input.getSourceGlobalStreamId().get_streamId();
+        } else if (isPair(input)) {
+            value = Pair.of(input.getValue(0), input.getValue(1));
+            stream = input.getSourceStreamId();
+        } else {
+            value = input.getValue(0);
+            stream = input.getSourceStreamId();
+        }
+        return Pair.of(value, stream);
+    }
+
+    void processAndAck(Tuple input) {
+        RefCountedTuple refCountedTuple = new RefCountedTuple(input);
+        setAnchor(refCountedTuple);
+        if (isEventTimestamp()) {
+            setEventTimestamp(input.getLongByField(getTimestampField()));
+        }
+        Pair<Object, String> valueAndStream = getValueAndStream(input);
+        process(valueAndStream.getFirst(), valueAndStream.getSecond());
+        ack(refCountedTuple);
+    }
+
+    void process(Object value, String sourceStreamId) {
+        LOG.debug("Process value {}, sourceStreamId {}", value, sourceStreamId);
+        Collection<ProcessorNode> initialProcessors = streamToInitialProcessors.get(sourceStreamId);
+        for (ProcessorNode processorNode : initialProcessors) {
+            Processor processor = processorNode.getProcessor();
+            if (StreamUtil.isPunctuation(value)) {
+                if (shouldPunctuate(processorNode, sourceStreamId)) {
+                    processor.punctuate(null);
+                    clearPunctuationState(processorNode);
+                }
+            } else {
+                processor.execute(value, sourceStreamId);
+            }
+        }
+    }
+
+    void setStreamToInitialProcessors(Multimap<String, ProcessorNode> streamToInitialProcessors) {
+        this.streamToInitialProcessors = streamToInitialProcessors;
+    }
+
+    void addStreamToInitialProcessors(Multimap<String, ProcessorNode> streamToInitialProcessors) {
+        this.streamToInitialProcessors.putAll(streamToInitialProcessors);
+    }
+
+    Set<String> getInitialStreams() {
+        return streamToInitialProcessors.keySet();
+    }
+
+    void setTimestampField(String fieldName) {
+        timestampField = fieldName;
+    }
+
+    boolean isEventTimestamp() {
+        return timestampField != null;
+    }
+
+    void setEventTimestamp(long timestamp) {
+        for (EmittingProcessorContext ctx : emittingProcessorContexts) {
+            ctx.setEventTimestamp(timestamp);
+        }
+    }
+
+    private String getTimestampField() {
+        return timestampField;
+    }
+
+    // if there are no windowed/batched processors, we would ack immediately
+    private void ack(RefCountedTuple tuple) {
+        if (tuple.shouldAck()) {
+            LOG.debug("ACKing tuple {}", tuple);
+            outputCollector.ack(tuple.tuple());
+            tuple.setAcked();
+        }
+    }
+
+    private ProcessorContext createEmittingContext(ProcessorNode processorNode) {
+        List<EmittingProcessorContext> emittingContexts = new ArrayList<>();
+        for (String stream : processorNode.getOutputStreams()) {
+            EmittingProcessorContext emittingContext = new EmittingProcessorContext(processorNode, outputCollector, stream);
+            if (StreamUtil.isSinkStream(stream)) {
+                emittingContext.setEmitPunctuation(false);
+            }
+            emittingContexts.add(emittingContext);
+        }
+        emittingProcessorContexts.addAll(emittingContexts);
+        outgoingProcessors.add(processorNode);
+        return new ChainedProcessorContext(processorNode, emittingContexts);
+    }
+
+    private boolean hasOutgoingChild(ProcessorNode processorNode, Set<ProcessorNode> boltChildren) {
+        for (Node child : getChildNodes(processorNode)) {
+            if ((child instanceof ProcessorNode && !boltChildren.contains(child))
+                    || child instanceof SinkNode) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    private Set<Node> getChildNodes(Node node) {
+        Set<Node> children = new HashSet<>();
+        for (Node child : StreamUtil.<Node>getChildren(graph, node)) {
+            if (child instanceof WindowNode || child instanceof PartitionNode) {
+                children.addAll(getChildNodes(child));
+            } else {
+                children.add(child);
+            }
+        }
+        return children;
+    }
+
+    // if we received punctuation from all parent windowed streams
+    private boolean shouldPunctuate(ProcessorNode processorNode, String sourceStreamId) {
+        if (processorNode.getWindowedParentStreams().size() <= 1) {
+            return true;
+        }
+        Set<String> receivedStreams = punctuationState.get(processorNode);
+        if (receivedStreams == null) {
+            receivedStreams = new HashSet<>();
+            punctuationState.put(processorNode, receivedStreams);
+        }
+        receivedStreams.add(sourceStreamId);
+        return receivedStreams.equals(processorNode.getWindowedParentStreams());
+    }
+
+    private void clearPunctuationState(ProcessorNode processorNode) {
+        Set<String> state = punctuationState.get(processorNode);
+        if (state != null) {
+            state.clear();
+        }
+    }
+
+    private boolean isPair(Tuple input) {
+        return input.size() == (timestampField == null ? 2 : 3);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/ProcessorNode.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/ProcessorNode.java b/storm-core/src/jvm/org/apache/storm/streams/ProcessorNode.java
new file mode 100644
index 0000000..4771f4f
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/ProcessorNode.java
@@ -0,0 +1,81 @@
+/**
+ * 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.processors.BatchProcessor;
+import org.apache.storm.streams.processors.Processor;
+import org.apache.storm.streams.processors.ProcessorContext;
+import org.apache.storm.tuple.Fields;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Node that wraps a processor in the stream.
+ */
+public class ProcessorNode extends Node {
+    private final Processor<?> processor;
+    private final boolean isBatch;
+    private boolean windowed;
+    // Windowed parent streams
+    private Set<String> windowedParentStreams = Collections.emptySet();
+
+    public ProcessorNode(Processor<?> processor, String outputStream, Fields outputFields) {
+        super(outputStream, outputFields);
+        this.isBatch = processor instanceof BatchProcessor;
+        this.processor = processor;
+    }
+
+    public Processor<?> getProcessor() {
+        return processor;
+    }
+
+    public boolean isWindowed() {
+        return windowed;
+    }
+
+    public boolean isBatch() {
+        return isBatch;
+    }
+
+    public void setWindowed(boolean windowed) {
+        this.windowed = windowed;
+    }
+
+    public Set<String> getWindowedParentStreams() {
+        return Collections.unmodifiableSet(windowedParentStreams);
+    }
+
+    void initProcessorContext(ProcessorContext context) {
+        processor.init(context);
+    }
+
+    void setWindowedParentStreams(Set<String> windowedParentStreams) {
+        this.windowedParentStreams = new HashSet<>(windowedParentStreams);
+    }
+
+    @Override
+    public String toString() {
+        return "ProcessorNode{" +
+                "processor=" + processor +
+                ", windowed=" + windowed +
+                ", windowedParentStreams=" + windowedParentStreams +
+                "} " + super.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/RefCountedTuple.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/RefCountedTuple.java b/storm-core/src/jvm/org/apache/storm/streams/RefCountedTuple.java
new file mode 100644
index 0000000..e58bcc5
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/RefCountedTuple.java
@@ -0,0 +1,64 @@
+/**
+ * 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.Tuple;
+
+/**
+ * Provides reference counting of tuples. Used when operations that operate
+ * on a batch of tuples are involved (e.g. aggregation, join etc).
+ * The input tuples are acked once the result is emitted downstream.
+ */
+public class RefCountedTuple {
+    private int count = 0;
+    private final Tuple tuple;
+    private boolean acked;
+
+    RefCountedTuple(Tuple tuple) {
+        this.tuple = tuple;
+        this.acked = false;
+    }
+
+    public boolean shouldAck() {
+        return count == 0 && !acked;
+    }
+
+    public void increment() {
+        ++count;
+    }
+
+    public void decrement() {
+        --count;
+    }
+
+    public Tuple tuple() {
+        return tuple;
+    }
+
+    public void setAcked() {
+        acked = true;
+    }
+
+    @Override
+    public String toString() {
+        return "RefCountedTuple{" +
+                "count=" + count +
+                ", tuple=" + tuple +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/SinkNode.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/SinkNode.java b/storm-core/src/jvm/org/apache/storm/streams/SinkNode.java
new file mode 100644
index 0000000..d95ab6b
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/SinkNode.java
@@ -0,0 +1,44 @@
+/**
+ * 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.IBasicBolt;
+import org.apache.storm.topology.IComponent;
+import org.apache.storm.topology.IRichBolt;
+import org.apache.storm.utils.Utils;
+
+/**
+ * Sink node holds IRich or IBasic bolts that are passed
+ * via the {@code Stream#to()} api.
+ */
+class SinkNode extends Node {
+    private final IComponent bolt;
+
+    SinkNode(IComponent bolt) {
+        super(Utils.DEFAULT_STREAM_ID, getOutputFields(bolt, Utils.DEFAULT_STREAM_ID));
+        if (bolt instanceof IRichBolt || bolt instanceof IBasicBolt) {
+            this.bolt = bolt;
+        } else {
+            throw new IllegalArgumentException("Should be an IRichBolt or IBasicBolt");
+        }
+    }
+
+    IComponent getBolt() {
+        return bolt;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/SpoutNode.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/SpoutNode.java b/storm-core/src/jvm/org/apache/storm/streams/SpoutNode.java
new file mode 100644
index 0000000..4784514
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/SpoutNode.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;
+
+import org.apache.storm.topology.IRichSpout;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.utils.Utils;
+
+/**
+ * A spout node wraps an {@link IRichSpout}.
+ */
+class SpoutNode extends Node {
+    private final IRichSpout spout;
+
+    SpoutNode(IRichSpout spout) {
+        super(Utils.DEFAULT_STREAM_ID, getOutputFields(spout, Utils.DEFAULT_STREAM_ID));
+        if (outputFields.size() == 0) {
+            throw new IllegalArgumentException("Spout " + spout + " does not declare any fields" +
+                    "for the stream '" + Utils.DEFAULT_STREAM_ID + "'");
+        }
+        this.spout = spout;
+    }
+
+    IRichSpout getSpout() {
+        return spout;
+    }
+
+    @Override
+    void addOutputStream(String streamId) {
+        throw new UnsupportedOperationException("Cannot add output streams to a spout node");
+    }
+
+}


[03/12] storm git commit: [STORM-1961] Stream api for storm core use cases

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/StatefulProcessorBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/StatefulProcessorBolt.java b/storm-core/src/jvm/org/apache/storm/streams/StatefulProcessorBolt.java
new file mode 100644
index 0000000..c123658
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/StatefulProcessorBolt.java
@@ -0,0 +1,116 @@
+/**
+ * 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.state.KeyValueState;
+import org.apache.storm.streams.processors.StatefulProcessor;
+import org.apache.storm.streams.processors.UpdateStateByKeyProcessor;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseStatefulBolt;
+import org.apache.storm.tuple.Tuple;
+import org.jgrapht.DirectedGraph;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Stream bolt that executes stateful operations like update state and state query.
+ */
+class StatefulProcessorBolt<K, V> extends BaseStatefulBolt<KeyValueState<K, V>> implements StreamBolt {
+    private final ProcessorBoltDelegate delegate;
+    // can be UpdateStateByKey or StateQuery processors
+    private final Set<StatefulProcessor<K, V>> statefulProcessors;
+
+    StatefulProcessorBolt(String boltId, DirectedGraph<Node, Edge> graph, List<ProcessorNode> nodes) {
+        delegate = new ProcessorBoltDelegate(boltId, graph, nodes);
+        statefulProcessors = getStatefulProcessors(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 initState(KeyValueState<K, V> state) {
+        for (StatefulProcessor<K, V> statefulProcessor : statefulProcessors) {
+            statefulProcessor.initState(state);
+        }
+    }
+
+    @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<String, ProcessorNode> streamToInitialProcessors) {
+        delegate.setStreamToInitialProcessors(streamToInitialProcessors);
+    }
+
+    public List<ProcessorNode> getNodes() {
+        return delegate.getNodes();
+    }
+
+    void addStreamToInitialProcessors(Multimap<String, ProcessorNode> streamToInitialProcessors) {
+        delegate.addStreamToInitialProcessors(streamToInitialProcessors);
+    }
+
+    void addNodes(List<ProcessorNode> nodes) {
+        delegate.addNodes(nodes);
+        statefulProcessors.addAll(getStatefulProcessors(nodes));
+    }
+
+    @SuppressWarnings("unchecked")
+    private Set<StatefulProcessor<K, V>> getStatefulProcessors(List<ProcessorNode> nodes) {
+        Set<StatefulProcessor<K, V>> statefulProcessors = new HashSet<>();
+        int updateStateByKeyCount = 0;
+        for (ProcessorNode node : nodes) {
+            if (node.getProcessor() instanceof StatefulProcessor) {
+                statefulProcessors.add((StatefulProcessor<K, V>) node.getProcessor());
+                if (node.getProcessor() instanceof UpdateStateByKeyProcessor) {
+                    if (++updateStateByKeyCount > 1) {
+                        throw new IllegalArgumentException("Cannot have more than one updateStateByKey processor " +
+                                "in a StatefulProcessorBolt");
+                    }
+                }
+
+            }
+        }
+        return statefulProcessors;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/Stream.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/Stream.java b/storm-core/src/jvm/org/apache/storm/streams/Stream.java
new file mode 100644
index 0000000..e50e7a2
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/Stream.java
@@ -0,0 +1,393 @@
+/**
+ * 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.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.IdentityFunction;
+import org.apache.storm.streams.operations.PairFlatMapFunction;
+import org.apache.storm.streams.operations.PairFunction;
+import org.apache.storm.streams.operations.Predicate;
+import org.apache.storm.streams.operations.PrintConsumer;
+import org.apache.storm.streams.operations.Reducer;
+import org.apache.storm.streams.processors.AggregateProcessor;
+import org.apache.storm.streams.processors.BranchProcessor;
+import org.apache.storm.streams.processors.FilterProcessor;
+import org.apache.storm.streams.processors.FlatMapProcessor;
+import org.apache.storm.streams.processors.ForEachProcessor;
+import org.apache.storm.streams.processors.MapProcessor;
+import org.apache.storm.streams.processors.PeekProcessor;
+import org.apache.storm.streams.processors.Processor;
+import org.apache.storm.streams.processors.ReduceProcessor;
+import org.apache.storm.streams.processors.StateQueryProcessor;
+import org.apache.storm.streams.windowing.Window;
+import org.apache.storm.topology.IBasicBolt;
+import org.apache.storm.topology.IRichBolt;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.utils.Utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Represents a stream of values.
+ *
+ * @param <T> the type of the value
+ */
+public class Stream<T> {
+    protected static final Fields KEY = new Fields("key");
+    protected static final Fields VALUE = new Fields("value");
+    protected static final Fields KEY_VALUE = new Fields("key", "value");
+    // the stream builder
+    protected final StreamBuilder streamBuilder;
+    // the current node
+    protected final Node node;
+    // the stream id from node's output stream(s) that this stream represents
+    protected final String stream;
+
+    Stream(StreamBuilder streamBuilder, Node node) {
+        this(streamBuilder, node, node.getOutputStreams().iterator().next());
+    }
+
+    private Stream(StreamBuilder streamBuilder, Node node, String stream) {
+        this.streamBuilder = streamBuilder;
+        this.node = node;
+        this.stream = stream;
+    }
+
+    /**
+     * Returns a stream consisting of the elements of this stream that matches the given filter.
+     *
+     * @param predicate the predicate to apply to each element to determine if it should be included
+     * @return the new stream
+     */
+    public Stream<T> filter(Predicate<? super T> predicate) {
+        return new Stream<>(streamBuilder, addProcessorNode(new FilterProcessor<>(predicate), VALUE));
+    }
+
+    /**
+     * Returns a stream consisting of the result of applying the given mapping function to the values of this stream.
+     *
+     * @param function a mapping function to be applied to each value in this stream.
+     * @return the new stream
+     */
+    public <R> Stream<R> map(Function<? super T, ? extends R> function) {
+        return new Stream<>(streamBuilder, addProcessorNode(new MapProcessor<>(function), VALUE));
+    }
+
+    /**
+     * Returns a stream of key-value pairs by applying a {@link PairFunction} on each value of this stream.
+     *
+     * @param function the mapping function to be applied to each value in this stream
+     * @param <K>      the key type
+     * @param <V>      the value type
+     * @return the new stream of key-value pairs
+     */
+    public <K, V> PairStream<K, V> mapToPair(PairFunction<T, K, V> function) {
+        return new PairStream<>(streamBuilder, addProcessorNode(new MapProcessor<>(function), KEY_VALUE));
+    }
+
+    /**
+     * Returns a stream consisting of the results of replacing each value of this stream with the contents
+     * produced by applying the provided mapping function to each value. This has the effect of applying
+     * a one-to-many transformation to the values of the stream, and then flattening the resulting elements
+     * into a new stream.
+     *
+     * @param function a mapping function to be applied to each value in this stream which produces new values.
+     * @return the new stream
+     */
+    public <R> Stream<R> flatMap(FlatMapFunction<T, R> function) {
+        return new Stream<>(streamBuilder, addProcessorNode(new FlatMapProcessor<>(function), VALUE));
+    }
+
+    /**
+     * Returns a stream consisting of the results of replacing each value of this stream with the key-value pairs
+     * produced by applying the provided mapping function to each value.
+     *
+     * @param function the mapping function to be applied to each value in this stream which produces new key-value pairs.
+     * @param <K>      the key type
+     * @param <V>      the value type
+     * @return the new stream of key-value pairs
+     * @see #flatMap(FlatMapFunction)
+     * @see #mapToPair(PairFunction)
+     */
+    public <K, V> PairStream<K, V> flatMapToPair(PairFlatMapFunction<T, K, V> function) {
+        return new PairStream<>(streamBuilder, addProcessorNode(new FlatMapProcessor<>(function), KEY_VALUE));
+    }
+
+    /**
+     * Returns a new stream consisting of the elements that fall within the window as specified by the window parameter.
+     * The {@link Window} specification could be used to specify sliding or tumbling windows based on
+     * time duration or event count. For example,
+     * <pre>
+     * // time duration based sliding window
+     * stream.window(SlidingWindows.of(Duration.minutes(10), Duration.minutes(1));
+     *
+     * // count based sliding window
+     * stream.window(SlidingWindows.of(Count.(10), Count.of(2)));
+     *
+     * // time duration based tumbling window
+     * stream.window(TumblingWindows.of(Duration.seconds(10));
+     * </p>
+     *
+     * @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<T> 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<T> 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.
+     * <p>
+     * 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.
+     * </p>
+     *
+     * @param aggregator the aggregator
+     * @param <R>        the result type
+     * @return the new stream
+     */
+    public <R> Stream<R> 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.
+     * <p>
+     * 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.
+     * </p>
+     *
+     * @param reducer the reducer
+     * @return the new stream
+     */
+    public Stream<T> reduce(Reducer<T> 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<T> 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<T>[] branch(Predicate<T>... predicates) {
+        List<Stream<T>> childStreams = new ArrayList<>();
+        if (predicates.length > 0) {
+            BranchProcessor<T> branchProcessor = new BranchProcessor<>();
+            Node branchNode = addProcessorNode(branchProcessor, VALUE);
+            for (Predicate<T> 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<T>[]) new Stream[childStreams.size()]);
+    }
+
+    /**
+     * Print the values in this stream.
+     */
+    public void print() {
+        forEach(new PrintConsumer<T>());
+    }
+
+    /**
+     * 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.
+     * <p>
+     * <b>Note:</b> This would provide guarantees only based on what the bolt provides.
+     * </p>
+     *
+     * @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}.
+     * <p>
+     * <b>Note:</b> This would provide guarantees only based on what the bolt provides.
+     * </p>
+     *
+     * @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.
+     * <p>
+     * <b>Note:</b> This would provide guarantees only based on what the bolt provides.
+     * </p>
+     *
+     * @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}.
+     * <p>
+     * <b>Note:</b> This would provide guarantees only based on what the bolt provides.
+     * </p>
+     *
+     * @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 <V>         the value type
+     * @return the result stream
+     */
+    public <V> PairStream<T, V> stateQuery(StreamState<T, V> 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<T> global() {
+        Node partitionNode = addNode(new PartitionNode(stream, node.getOutputFields(), GroupingInfo.global()));
+        return new Stream<>(streamBuilder, partitionNode);
+    }
+
+    private Stream<T> all() {
+        Node partitionNode = addNode(new PartitionNode(stream, node.getOutputFields(), GroupingInfo.all()));
+        return new Stream<>(streamBuilder, partitionNode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/StreamBolt.java
----------------------------------------------------------------------
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 0000000..6170ea8
--- /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);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java
----------------------------------------------------------------------
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 0000000..e19a0c6
--- /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<Node, Edge> graph;
+    private final Table<Node, String, GroupingInfo> nodeGroupingInfo = HashBasedTable.create();
+    private final Map<Node, WindowNode> windowInfo = new HashMap<>();
+    private final List<ProcessorNode> curGroup = new ArrayList<>();
+    private int statefulProcessorCount = 0;
+    private final Map<StreamBolt, BoltDeclarer> 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<Tuple> 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<Tuple> 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 <T>         the type of values in the resultant stream
+     * @return the new stream
+     */
+    public <T> Stream<T> newStream(IRichSpout spout, TupleValueMapper<T> 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 <T>         the type of values in the resultant stream
+     * @return the new stream
+     */
+    public <T> Stream<T> newStream(IRichSpout spout, TupleValueMapper<T> 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 <K>             the key type
+     * @param <V>             the value type
+     * @return the new stream of key-value pairs
+     */
+    public <K, V> PairStream<K, V> newStream(IRichSpout spout, PairValueMapper<K, V> 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 <K>             the key type
+     * @param <V>             the value type
+     * @return the new stream of key-value pairs
+     */
+    public <K, V> PairStream<K, V> newStream(IRichSpout spout, PairValueMapper<K, V> 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<Node, Edge> 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<Node> queue() {
+        // min-heap
+        return new PriorityQueue<>(new Comparator<Node>() {
+            @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<ProcessorNode> initialNodes = initialProcessors(
+                    curGroup.isEmpty() ? Collections.singletonList(processorNode) : curGroup);
+            Set<Window<?, ?>> 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.<Node>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<Node> 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<Node> parentNodes(Node curNode) {
+        Set<Node> nodes = new HashSet<>();
+        for (Node parent : StreamUtil.<Node>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<ProcessorNode> initialProcessors = initialProcessors(curGroup);
+        Set<Window<?, ?>> 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<ProcessorNode> processorNodes) {
+        for (ProcessorNode node : processorNodes) {
+            if (node.getProcessor() instanceof StatefulProcessor) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    private int getParallelism() {
+        Set<Integer> parallelisms = new HashSet<>(Collections2.transform(curGroup, new Function<ProcessorNode, Integer>() {
+            @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<Window<?, ?>> getWindowParams(Set<ProcessorNode> initialProcessors) {
+        Set<WindowNode> windowNodes = new HashSet<>();
+        Set<Node> 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<Window<?, ?>> windowParams = new HashSet<>();
+        if (!windowNodes.isEmpty()) {
+            windowParams.addAll(new HashSet<>(Collections2.transform(windowNodes, new Function<WindowNode, Window<?, ?>>() {
+                @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<ProcessorNode> 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<ProcessorNode> 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<ProcessorNode> 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<String> getWindowedParentStreams(ProcessorNode processorNode) {
+        Set<String> 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<String>() {
+                        @Override
+                        public boolean apply(String input) {
+                            return !StreamUtil.isSinkStream(input);
+                        }
+                    }));
+                }
+            }
+        }
+        return res;
+    }
+
+    private Multimap<String, ProcessorNode> wireBolt(List<ProcessorNode> curGroup,
+                                                     BoltDeclarer boltDeclarer,
+                                                     Set<ProcessorNode> initialProcessors) {
+        LOG.debug("Wiring bolt with boltDeclarer {}, curGroup {}, initialProcessors {}, nodeGroupingInfo {}",
+                boltDeclarer, curGroup, initialProcessors, nodeGroupingInfo);
+        Multimap<String, ProcessorNode> streamToInitialProcessor = ArrayListMultimap.create();
+        Set<ProcessorNode> 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<ProcessorNode> initialProcessors(List<ProcessorNode> curGroup) {
+        Set<ProcessorNode> nodes = new HashSet<>();
+        Set<ProcessorNode> 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.<Node>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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/StreamState.java
----------------------------------------------------------------------
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 0000000..a4633f7
--- /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 <K> the key type
+ * @param <V> the value type
+ */
+public class StreamState<K, V> implements Serializable {
+    private final transient PairStream<K, V> stream;
+
+    StreamState(PairStream<K, V> stream) {
+        this.stream = stream;
+    }
+
+    public PairStream<K, V> toPairStream() {
+        return stream;
+    }
+
+    ProcessorNode getUpdateStateNode() {
+        return (ProcessorNode) stream.node;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/StreamUtil.java
----------------------------------------------------------------------
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 0000000..0531ff6
--- /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 <T> List<T> getParents(DirectedGraph<Node, Edge> graph, Node node) {
+        List<Edge> incoming = new ArrayList<>(graph.incomingEdgesOf(node));
+        List<T> ret = new ArrayList<>();
+        for (Edge e : incoming) {
+            ret.add((T) e.getSource());
+        }
+        return ret;
+    }
+
+    @SuppressWarnings("unchecked")
+    public static <T> List<T> getChildren(DirectedGraph<Node, Edge> graph, Node node) {
+        List<Edge> outgoing = new ArrayList<>(graph.outgoingEdgesOf(node));
+        List<T> 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);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/StreamsEdgeFactory.java
----------------------------------------------------------------------
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 0000000..0078690
--- /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<Node, Edge>, Serializable {
+    @Override
+    public Edge createEdge(Node sourceVertex, Node targetVertex) {
+        return new Edge(sourceVertex, targetVertex);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/Tuple3.java
----------------------------------------------------------------------
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 0000000..77973f2
--- /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 <T1> the type of the first element
+ * @param <T2> the type of the second element
+ * @param <T3> the type of the third element
+ */
+public class Tuple3<T1, T2, T3> {
+    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 + ")";
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/UniqueIdGen.java
----------------------------------------------------------------------
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 0000000..3cbd141
--- /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;
+    }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/WindowNode.java
----------------------------------------------------------------------
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 0000000..a0e831a
--- /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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/WindowedProcessorBolt.java
----------------------------------------------------------------------
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 0000000..3971346
--- /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<Node, Edge> graph,
+                          List<ProcessorNode> 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<Object, String> 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<String, ProcessorNode> streamToInitialProcessors) {
+        delegate.setStreamToInitialProcessors(streamToInitialProcessors);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/operations/Aggregator.java
----------------------------------------------------------------------
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 0000000..e3feaf4
--- /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 <T> the original value type
+ * @param <R> the aggregated value type
+ */
+public interface Aggregator<T, R> 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);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/operations/Consumer.java
----------------------------------------------------------------------
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 0000000..84653ab
--- /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 <T> the type of the input argument
+ */
+public interface Consumer<T> extends Operation {
+    /**
+     * Performs an operation on the given argument.
+     *
+     * @param input the input
+     */
+    void accept(T input);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/operations/FlatMapFunction.java
----------------------------------------------------------------------
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 0000000..bcacd08
--- /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 <T> the type of the input to the function
+ * @param <R> the result type. An iterable of this type is returned from this function
+ */
+public interface FlatMapFunction<T, R> extends Function<T, Iterable<R>> {
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/operations/Function.java
----------------------------------------------------------------------
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 0000000..7cef0a6
--- /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 <T> the type of the input to the function
+ * @param <R> the type of the result of the function
+ */
+public interface Function<T, R> extends Operation {
+    /**
+     * Applies this function to the given argument.
+     *
+     * @param input the input to the function
+     * @return the function result
+     */
+    R apply(T input);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/operations/IdentityFunction.java
----------------------------------------------------------------------
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 0000000..abb9327
--- /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 <T> the type of the input
+ */
+public class IdentityFunction<T> implements Function<T, T> {
+
+    @Override
+    public T apply(T input) {
+        return input;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/operations/Operation.java
----------------------------------------------------------------------
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 0000000..77dbe1e
--- /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 {
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/operations/PairFlatMapFunction.java
----------------------------------------------------------------------
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 0000000..376c1ba
--- /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 <T> the type of the input to the function
+ * @param <K> the key type of the key-value pairs produced as a result
+ * @param <V> the value type of the key-value pairs produced as a result
+ */
+public interface PairFlatMapFunction<T, K, V> extends FlatMapFunction<T, Pair<K, V>> {
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/operations/PairFunction.java
----------------------------------------------------------------------
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 0000000..b153ff1
--- /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 <T> the type of the input to the function
+ * @param <K> the key type
+ * @param <V> the value type
+ */
+public interface PairFunction<T, K, V> extends Function<T, Pair<K,V>> {
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/operations/PairValueJoiner.java
----------------------------------------------------------------------
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 0000000..ca81101
--- /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 <V1> the type of the first value
+ * @param <V2> the type of the second value
+ */
+public class PairValueJoiner<V1, V2> implements ValueJoiner<V1, V2, Pair<V1, V2>> {
+    /**
+     * 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<V1, V2> apply(V1 value1, V2 value2) {
+        return Pair.of(value1, value2);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/operations/Predicate.java
----------------------------------------------------------------------
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 0000000..ae1be2d
--- /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 <T> the value type
+ */
+public interface Predicate<T> 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);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/operations/PrintConsumer.java
----------------------------------------------------------------------
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 0000000..d301dbf
--- /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 <T> the value type
+ */
+public class PrintConsumer<T> implements Consumer<T> {
+    @Override
+    public void accept(T input) {
+        System.out.println(input);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e251573d/storm-core/src/jvm/org/apache/storm/streams/operations/Reducer.java
----------------------------------------------------------------------
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 0000000..04ee70d
--- /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 <T> the type of the arguments and the result
+ */
+public interface Reducer<T> 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);
+}


[12/12] storm git commit: STORM-1961: CHANGELOG

Posted by ka...@apache.org.
STORM-1961: CHANGELOG


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/b5f02d4e
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/b5f02d4e
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/b5f02d4e

Branch: refs/heads/master
Commit: b5f02d4e4cac577454c379ca4e150c7ae47307be
Parents: 00b0fd7
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Tue Feb 7 10:27:56 2017 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Tue Feb 7 10:27:56 2017 +0900

----------------------------------------------------------------------
 CHANGELOG.md | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/b5f02d4e/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 3c41ba3..8dc0c8a 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
-\ufeff## 2.0.0
+## 2.0.0
+ * STORM-1961: Stream api for storm core use cases
  * STORM-2327: Introduce ConfigurableTopology
  * STORM-2323: Precondition for Leader Nimbus should check all topology blobs and also corresponding dependencies.
  * STORM-2305: STORM-2279 calculates task index different from grouper code


[10/12] storm git commit: [STORM-1961] Addressed review comments and a few other refactoring

Posted by ka...@apache.org.
[STORM-1961] Addressed review comments and a few other refactoring


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/37ee12cc
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/37ee12cc
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/37ee12cc

Branch: refs/heads/master
Commit: 37ee12ccd9a0465dc6cd6d2f9fb59929a798b6b5
Parents: a07039f
Author: Arun Mahadevan <ar...@apache.org>
Authored: Tue Jan 24 17:35:32 2017 +0530
Committer: Arun Mahadevan <ar...@apache.org>
Committed: Tue Jan 24 22:25:21 2017 +0530

----------------------------------------------------------------------
 .../storm/starter/streams/AggregateExample.java | 17 ++++----
 .../storm/starter/streams/BranchExample.java    |  9 ++--
 .../streams/GroupByKeyAndWindowExample.java     | 23 ++++++----
 .../storm/starter/streams/JoinExample.java      |  9 ++--
 .../starter/streams/StateQueryExample.java      | 16 ++++---
 .../starter/streams/StatefulWordCount.java      | 15 ++++---
 .../starter/streams/TypedTupleExample.java      |  9 ++--
 .../starter/streams/WindowedWordCount.java      |  9 ++--
 .../storm/starter/streams/WordCountToBolt.java  |  9 ++--
 .../src/jvm/org/apache/storm/streams/Pair.java  | 28 +++++++-----
 .../org/apache/storm/streams/PairStream.java    |  3 +-
 .../jvm/org/apache/storm/streams/Stream.java    |  5 ++-
 .../streams/operations/aggregators/LongSum.java | 45 ++++++++++++++++++++
 .../streams/operations/aggregators/Sum.java     | 45 --------------------
 .../processors/AggregateByKeyProcessor.java     |  5 ++-
 .../streams/processors/AggregateProcessor.java  |  4 +-
 .../storm/streams/processors/BaseProcessor.java |  8 ++--
 .../MergeAggregateByKeyProcessor.java           |  2 +-
 .../processors/MergeAggregateProcessor.java     |  2 +-
 .../storm/streams/processors/Processor.java     |  2 +-
 .../streams/processors/ProcessorContext.java    |  4 +-
 .../processors/ReduceByKeyProcessor.java        |  6 +--
 .../streams/processors/ReduceProcessor.java     |  2 +-
 .../apache/storm/streams/ProcessorBoltTest.java |  4 +-
 24 files changed, 149 insertions(+), 132 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java
index 91dfadb..7467634 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java
@@ -60,11 +60,10 @@ public class AggregateExample {
             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();
+            try (LocalCluster cluster = new LocalCluster();
+                 LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) {
+                Utils.sleep(60_000);
+            }
         }
     }
 
@@ -76,21 +75,21 @@ public class AggregateExample {
 
         @Override
         public Pair<Integer, Integer> apply(Pair<Integer, Integer> sumAndCount, Integer value) {
-            return Pair.of(sumAndCount.getFirst() + value, sumAndCount.getSecond() + 1);
+            return Pair.of(sumAndCount._1 + value, sumAndCount._2 + 1);
         }
 
         @Override
         public Pair<Integer, Integer> merge(Pair<Integer, Integer> sumAndCount1, Pair<Integer, Integer> sumAndCount2) {
             System.out.println("Merge " + sumAndCount1 + " and " + sumAndCount2);
             return Pair.of(
-                    sumAndCount1.getFirst() + sumAndCount2.getFirst(),
-                    sumAndCount1.getSecond() + sumAndCount2.getSecond()
+                    sumAndCount1._1 + sumAndCount2._1,
+                    sumAndCount1._2 + sumAndCount2._2
             );
         }
 
         @Override
         public Double result(Pair<Integer, Integer> sumAndCount) {
-            return (double) sumAndCount.getFirst()/sumAndCount.getSecond();
+            return (double) sumAndCount._1/sumAndCount._2;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/BranchExample.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
index f5400a5..027b432 100644
--- 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
@@ -61,11 +61,10 @@ public class BranchExample {
             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();
+            try (LocalCluster cluster = new LocalCluster();
+                 LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) {
+                Utils.sleep(60_000);
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/GroupByKeyAndWindowExample.java
----------------------------------------------------------------------
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
index 6b505bd..dd7e97f 100644
--- 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
@@ -52,6 +52,9 @@ public class GroupByKeyAndWindowExample {
                 /*
                  * The elements having the same key within the window will be grouped
                  * together and the corresponding values will be merged.
+                 *
+                 * The result is a PairStream<String, Iterable<Double>> with
+                 * 'stock symbol' as the key and 'stock prices' for that symbol within the window as the value.
                  */
                 .groupByKeyAndWindow(SlidingWindows.of(Count.of(6), Count.of(3)))
                 .print();
@@ -61,8 +64,11 @@ public class GroupByKeyAndWindowExample {
                 /*
                  * The elements having the same key within the window will be grouped
                  * together and their values will be reduced using the given reduce function.
+                 *
+                 * Here the result is a PairStream<String, Double> with
+                 * 'stock symbol' as the key and the maximum price for that symbol within the window as the value.
                  */
-                .reduceByKeyAndWindow((x, y) -> (x + y) / 2.0, SlidingWindows.of(Count.of(6), Count.of(3)))
+                .reduceByKeyAndWindow((x, y) -> x > y ? x : y, SlidingWindows.of(Count.of(6), Count.of(3)))
                 .print();
 
         Config config = new Config();
@@ -70,19 +76,18 @@ public class GroupByKeyAndWindowExample {
             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();
+            try (LocalCluster cluster = new LocalCluster();
+                 LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) {
+                Utils.sleep(60_000);
+            }
         }
     }
 
     private static class StockQuotes extends BaseRichSpout {
         private final List<List<Values>> 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))
+                Arrays.asList(new Values("AAPL", 105.0), new Values("GOOG", 790.0), new Values("FB", 130.0)),
+                Arrays.asList(new Values("AAPL", 102.0), new Values("GOOG", 788.0), new Values("FB", 128.0))
         );
         private SpoutOutputCollector collector;
         private int index = 0;
@@ -103,7 +108,7 @@ public class GroupByKeyAndWindowExample {
 
         @Override
         public void declareOutputFields(OutputFieldsDeclarer declarer) {
-            declarer.declare(new Fields("key", "val"));
+            declarer.declare(new Fields("symbol", "price"));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/JoinExample.java
----------------------------------------------------------------------
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
index 0b15615..4aa6253 100644
--- 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
@@ -71,11 +71,10 @@ public class JoinExample {
             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();
+            try (LocalCluster cluster = new LocalCluster();
+                 LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) {
+                Utils.sleep(60_000);
+            }
         }
 
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java
----------------------------------------------------------------------
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
index 6d6a4b3..ab6cac3 100644
--- 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
@@ -54,7 +54,14 @@ public class StateQueryExample {
     public static void main(String[] args) throws Exception {
         StreamBuilder builder = new StreamBuilder();
         StreamState<String, Long> ss = builder.newStream(new TestWordSpout(), new ValueMapper<String>(0), 2)
+                /*
+                 * Transform the stream of words to a stream of (word, 1) pairs
+                 */
                 .mapToPair(w -> Pair.of(w, 1))
+                /*
+                 *  Update the count in the state. Here the first argument 0L is the initial value for the count and
+                 *  the second argument is a function that increments the count for each value received.
+                 */
                 .updateStateByKey(0L, (count, val) -> count + 1);
 
         /*
@@ -77,11 +84,10 @@ public class StateQueryExample {
             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();
+            try (LocalCluster cluster = new LocalCluster();
+                 LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) {
+                Utils.sleep(60_000);
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java
----------------------------------------------------------------------
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
index ce7470d..ddd318a 100644
--- 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
@@ -60,9 +60,11 @@ public class StatefulWordCount {
                  */
                 .countByKey()
                 /*
-                 * update the word counts in the state
+                 * update the word counts in the state.
+                 * Here the first argument 0L is the initial value for the state
+                 * and the second argument is a function that adds the count to the current value in the state.
                  */
-                .updateStateByKey(0L, (x, y) -> x + y)
+                .updateStateByKey(0L, (state, count) -> state + count)
                  /*
                   * convert the state back to a stream and print the results
                   */
@@ -77,11 +79,10 @@ public class StatefulWordCount {
             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();
+            try (LocalCluster cluster = new LocalCluster();
+                 LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) {
+                Utils.sleep(60_000);
+            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java
index 193ad661..11e89bf 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java
@@ -54,11 +54,10 @@ public class TypedTupleExample {
             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();
+            try (LocalCluster cluster = new LocalCluster();
+                 LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) {
+                Utils.sleep(60_000);
+            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java
----------------------------------------------------------------------
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
index 0765a74..0f30b7c 100644
--- 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
@@ -70,11 +70,10 @@ public class WindowedWordCount {
             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();
+            try (LocalCluster cluster = new LocalCluster();
+                 LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) {
+                Utils.sleep(60_000);
+            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java
----------------------------------------------------------------------
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
index dd7923a..1c0aae1 100644
--- 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
@@ -71,11 +71,10 @@ public class WordCountToBolt {
             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();
+            try (LocalCluster cluster = new LocalCluster();
+                 LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) {
+                Utils.sleep(60_000);
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/storm-core/src/jvm/org/apache/storm/streams/Pair.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/Pair.java b/storm-core/src/jvm/org/apache/storm/streams/Pair.java
index e5eb792..26d53b7 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/Pair.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/Pair.java
@@ -26,8 +26,14 @@ import java.io.Serializable;
  * @param <T2> the type of the second value
  */
 public final class Pair<T1, T2> implements Serializable {
-    private final T1 first;
-    private final T2 second;
+    /**
+     * The first value
+     */
+    public final T1 _1;
+    /**
+     * The second value
+     */
+    public final T2 _2;
 
     /**
      * Constructs a new pair of values
@@ -36,8 +42,8 @@ public final class Pair<T1, T2> implements Serializable {
      * @param second the second value
      */
     private Pair(T1 first, T2 second) {
-        this.first = first;
-        this.second = second;
+        _1 = first;
+        _2 = second;
     }
 
     /**
@@ -46,7 +52,7 @@ public final class Pair<T1, T2> implements Serializable {
      * @return the first value
      */
     public T1 getFirst() {
-        return first;
+        return _1;
     }
 
     /**
@@ -55,7 +61,7 @@ public final class Pair<T1, T2> implements Serializable {
      * @return the second value
      */
     public T2 getSecond() {
-        return second;
+        return _2;
     }
 
     /**
@@ -78,20 +84,20 @@ public final class Pair<T1, T2> implements Serializable {
 
         Pair<?, ?> pair = (Pair<?, ?>) o;
 
-        if (first != null ? !first.equals(pair.first) : pair.first != null) return false;
-        return second != null ? second.equals(pair.second) : pair.second == null;
+        if (_1 != null ? !_1.equals(pair._1) : pair._1 != null) return false;
+        return _2 != null ? _2.equals(pair._2) : pair._2 == null;
 
     }
 
     @Override
     public int hashCode() {
-        int result = first != null ? first.hashCode() : 0;
-        result = 31 * result + (second != null ? second.hashCode() : 0);
+        int result = _1 != null ? _1.hashCode() : 0;
+        result = 31 * result + (_2 != null ? _2.hashCode() : 0);
         return result;
     }
 
     @Override
     public String toString() {
-        return "(" + first + ", " + second + ')';
+        return "(" + _1 + ", " + _2 + ')';
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/storm-core/src/jvm/org/apache/storm/streams/PairStream.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/PairStream.java b/storm-core/src/jvm/org/apache/storm/streams/PairStream.java
index 3c08a05..da332a5 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/PairStream.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/PairStream.java
@@ -376,7 +376,8 @@ public class PairStream<K, V> extends Stream<Pair<K, V>> {
      * @return the {@link StreamState} which can be used to query the state
      */
     public <R> StreamState<K, R> updateStateByKey(StateUpdater<? super V, ? extends R> stateUpdater) {
-        return partitionByKey().updateStateByKeyPartition(stateUpdater);
+        // repartition so that state query fields grouping works correctly. this can be optimized further
+        return partitionBy(KEY).updateStateByKeyPartition(stateUpdater);
     }
 
     private <R> StreamState<K, R> updateStateByKeyPartition(StateUpdater<? super V, ? extends R> stateUpdater) {

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/storm-core/src/jvm/org/apache/storm/streams/Stream.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/Stream.java b/storm-core/src/jvm/org/apache/storm/streams/Stream.java
index 087e760..272c954 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/Stream.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/Stream.java
@@ -278,8 +278,9 @@ public class Stream<T> {
      * 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.
-     *
+     * <p>
+     * <b>Note:</b> If none of the predicates match a value, that value is dropped.
+     * </p>
      * @param predicates the predicates
      * @return an array of result streams (branches) corresponding to the given predicates
      */

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/LongSum.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/LongSum.java b/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/LongSum.java
new file mode 100644
index 0000000..afc7ee7
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/LongSum.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.aggregators;
+
+import org.apache.storm.streams.operations.CombinerAggregator;
+
+/**
+ * Computes the long sum of the input values
+ */
+public class LongSum implements CombinerAggregator<Number, Long, Long> {
+    @Override
+    public Long init() {
+        return 0L;
+    }
+
+    @Override
+    public Long apply(Long aggregate, Number value) {
+        return value.longValue() + aggregate;
+    }
+
+    @Override
+    public Long merge(Long accum1, Long accum2) {
+        return accum1 + accum2;
+    }
+
+    @Override
+    public Long result(Long accum) {
+        return accum;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Sum.java
----------------------------------------------------------------------
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
deleted file mode 100644
index df11d99..0000000
--- a/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Sum.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * 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.CombinerAggregator;
-
-/**
- * Computes the long sum of the input values
- */
-public class Sum implements CombinerAggregator<Number, Long, Long> {
-    @Override
-    public Long init() {
-        return 0L;
-    }
-
-    @Override
-    public Long apply(Long aggregate, Number value) {
-        return value.longValue() + aggregate;
-    }
-
-    @Override
-    public Long merge(Long accum1, Long accum2) {
-        return accum1 + accum2;
-    }
-
-    @Override
-    public Long result(Long accum) {
-        return accum;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java
----------------------------------------------------------------------
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
index c10113b..3a7c812 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java
@@ -22,6 +22,7 @@ import org.apache.storm.streams.operations.CombinerAggregator;
 
 import java.util.HashMap;
 import java.util.Map;
+import java.util.function.Supplier;
 
 public class AggregateByKeyProcessor<K, V, A, R> extends BaseProcessor<Pair<K, V>> implements BatchProcessor {
     private final CombinerAggregator<V, A, R> aggregator;
@@ -47,9 +48,9 @@ public class AggregateByKeyProcessor<K, V, A, R> extends BaseProcessor<Pair<K, V
         }
         state.put(key, aggregator.apply(accumulator, val));
         if (emitAggregate) {
-            mayBeForwardAggUpdate(Pair.of(key, state.get(key)));
+            mayBeForwardAggUpdate(() -> Pair.of(key, state.get(key)));
         } else {
-            mayBeForwardAggUpdate(Pair.of(key, aggregator.result(state.get(key))));
+            mayBeForwardAggUpdate(() -> Pair.of(key, aggregator.result(state.get(key))));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java
----------------------------------------------------------------------
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
index d169345..d8b78ad 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java
@@ -40,9 +40,9 @@ public class AggregateProcessor<T, A, R> extends BaseProcessor<T> implements Bat
         }
         state = aggregator.apply(state, input);
         if (emitAggregate) {
-            mayBeForwardAggUpdate(state);
+            mayBeForwardAggUpdate(() -> state);
         } else {
-            mayBeForwardAggUpdate(aggregator.result(state));
+            mayBeForwardAggUpdate(() -> aggregator.result(state));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/storm-core/src/jvm/org/apache/storm/streams/processors/BaseProcessor.java
----------------------------------------------------------------------
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
index 3ea469c..8413a2a 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/processors/BaseProcessor.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/BaseProcessor.java
@@ -19,6 +19,8 @@ package org.apache.storm.streams.processors;
 
 import java.util.HashSet;
 import java.util.Set;
+import java.util.function.Function;
+import java.util.function.Supplier;
 
 import static org.apache.storm.streams.WindowNode.PUNCTUATION;
 
@@ -91,12 +93,12 @@ abstract class BaseProcessor<T> implements Processor<T> {
      * can use this to emit the partial results on each input
      * if they are operating in non-windowed mode.
      *
-     * @param result the result
+     * @param result the result function
      * @param <R>    the result type
      */
-    protected final <R> void mayBeForwardAggUpdate(R result) {
+    protected final <R> void mayBeForwardAggUpdate(Supplier<R> result) {
         if (!context.isWindowed()) {
-            context.forward(result);
+            context.forward(result.get());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateByKeyProcessor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateByKeyProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateByKeyProcessor.java
index 57ad845..a66c88a 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateByKeyProcessor.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateByKeyProcessor.java
@@ -40,7 +40,7 @@ public class MergeAggregateByKeyProcessor<K, V, A, R> extends BaseProcessor<Pair
             accumulator = aggregator.init();
         }
         state.put(key, aggregator.merge(accumulator, val));
-        mayBeForwardAggUpdate(Pair.of(key, aggregator.result(state.get(key))));
+        mayBeForwardAggUpdate(() -> Pair.of(key, aggregator.result(state.get(key))));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateProcessor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateProcessor.java
index 61b555b..9c5b1f0 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateProcessor.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateProcessor.java
@@ -33,7 +33,7 @@ public class MergeAggregateProcessor<T, A, R> extends BaseProcessor<A> implement
             state = aggregator.init();
         }
         state = aggregator.merge(state, input);
-        mayBeForwardAggUpdate(aggregator.result(state));
+        mayBeForwardAggUpdate(() -> aggregator.result(state));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java
index 3b4518d..83daf2a 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java
@@ -37,7 +37,7 @@ public interface Processor<T> extends Serializable {
     void init(ProcessorContext context);
 
     /**
-     * Executes some operation on the input and possibly emits some result.
+     * Executes some operations on the input and possibly emits some results.
      *
      * @param input    the input to be processed
      * @param streamId the source stream id from where the input is received

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java
index a69a0a9..45f58a1 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java
@@ -45,10 +45,10 @@ public interface ProcessorContext extends Serializable {
     <T> void forward(T input, String stream);
 
     /**
-     * Returns if the processing is in a windowed context and should wait for
+     * Returns true if the processing is in a windowed context and should wait for
      * punctuation before emitting results.
      *
-     * @return if this is a windowed context or not
+     * @return whether this is a windowed context or not
      */
     boolean isWindowed();
 

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceByKeyProcessor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceByKeyProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceByKeyProcessor.java
index eba0de6..f20c041 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceByKeyProcessor.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceByKeyProcessor.java
@@ -36,9 +36,9 @@ public class ReduceByKeyProcessor<K, V> extends BaseProcessor<Pair<K, V>> implem
         K key = input.getFirst();
         V val = input.getSecond();
         V agg = state.get(key);
-        agg = (agg == null) ? val : reducer.apply(agg, val);
-        state.put(key, agg);
-        mayBeForwardAggUpdate(Pair.of(key, agg));
+        final V res = (agg == null) ? val : reducer.apply(agg, val);
+        state.put(key, res);
+        mayBeForwardAggUpdate(() -> Pair.of(key, res));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java
index 0b90fb9..93badf2 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java
@@ -30,7 +30,7 @@ public class ReduceProcessor<T> extends BaseProcessor<T> implements BatchProcess
     @Override
     public void execute(T input) {
         agg = (agg == null) ? input : reducer.apply(agg, input);
-        mayBeForwardAggUpdate(agg);
+        mayBeForwardAggUpdate(() -> agg);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/storm/blob/37ee12cc/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java b/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java
index aa877f9..9380714 100644
--- a/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java
+++ b/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java
@@ -19,7 +19,7 @@ package org.apache.storm.streams;
 
 import com.google.common.collect.Multimap;
 import org.apache.storm.generated.GlobalStreamId;
-import org.apache.storm.streams.operations.aggregators.Sum;
+import org.apache.storm.streams.operations.aggregators.LongSum;
 import org.apache.storm.streams.processors.AggregateProcessor;
 import org.apache.storm.streams.processors.FilterProcessor;
 import org.apache.storm.streams.processors.Processor;
@@ -89,7 +89,7 @@ public class ProcessorBoltTest {
 
     @Test
     public void testAggResultAndAck() throws Exception {
-        setUpProcessorBolt(new AggregateProcessor<>(new Sum()), Collections.singleton("inputstream"), true, null);
+        setUpProcessorBolt(new AggregateProcessor<>(new LongSum()), Collections.singleton("inputstream"), true, null);
         bolt.execute(mockTuple2);
         bolt.execute(mockTuple3);
         bolt.execute(punctuation);


[07/12] storm git commit: [STORM-1961] A few fixes and refactoring

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java b/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java
index e19a0c6..7dff25d 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java
@@ -30,8 +30,10 @@ 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.Processor;
 import org.apache.storm.streams.processors.StateQueryProcessor;
 import org.apache.storm.streams.processors.StatefulProcessor;
+import org.apache.storm.streams.processors.UpdateStateByKeyProcessor;
 import org.apache.storm.streams.windowing.Window;
 import org.apache.storm.topology.BoltDeclarer;
 import org.apache.storm.topology.IBasicBolt;
@@ -195,18 +197,30 @@ public class StreamBuilder {
     }
 
     Node addNode(Node parent, Node child) {
-        return addNode(parent, child, parent.getParallelism(), parent.getOutputStreams().iterator().next());
+        return addNode(parent, child, parent.getOutputStreams().iterator().next(), parent.getParallelism());
     }
 
     Node addNode(Node parent, Node child, int parallelism) {
-        return addNode(parent, child, parallelism, parent.getOutputStreams().iterator().next());
+        return addNode(parent, child, parent.getOutputStreams().iterator().next(), parallelism);
+    }
+
+    // insert child in-between parent and its current child nodes
+    Node insert(Node parent, Node child) {
+        Node newChild = addNode(parent, child);
+        for (Edge edge : graph.outgoingEdgesOf(parent)) {
+            Node oldChild = edge.getTarget();
+            graph.removeEdge(parent, oldChild);
+            oldChild.removeParentStreams(parent);
+            addNode(newChild, oldChild);
+        }
+        return newChild;
     }
 
     Node addNode(Node parent, Node child, String parentStreamId) {
-        return addNode(parent, child, parent.getParallelism(), parentStreamId);
+        return addNode(parent, child, parentStreamId, parent.getParallelism());
     }
 
-    Node addNode(Node parent, Node child, int parallelism, String parentStreamId) {
+    Node addNode(Node parent, Node child, String parentStreamId, int parallelism) {
         graph.addVertex(child);
         graph.addEdge(parent, child);
         child.setParallelism(parallelism);
@@ -215,34 +229,62 @@ public class StreamBuilder {
         } else {
             child.addParentStream(parent, parentStreamId);
         }
+        if (!(child instanceof PartitionNode)) {
+            if (child.getGroupingInfo() != null) {
+                if (!child.getGroupingInfo().equals(parent.getGroupingInfo())) {
+                    throw new IllegalStateException("Trying to assign grouping info for node" +
+                            " with current grouping info: " + child.getGroupingInfo() +
+                            " to: " + parent.getGroupingInfo() + " Node: " + child);
+                }
+            } else {
+                child.setGroupingInfo(parent.getGroupingInfo());
+            }
+        }
+        if (!(child instanceof WindowNode) && !child.isWindowed()) {
+            child.setWindowed(parent.isWindowed());
+        }
         return child;
     }
 
     private PriorityQueue<Node> queue() {
         // min-heap
         return new PriorityQueue<>(new Comparator<Node>() {
+            /*
+             * Nodes in the descending order of priority.
+             * ProcessorNode has higher priority than partition and window nodes
+             * so that the topological order iterator will group as many processor nodes together as possible.
+             * UpdateStateByKeyProcessor has a higher priority than StateQueryProcessor so that StateQueryProcessor
+             * can be mapped to the same StatefulBolt that UpdateStateByKeyProcessor is part of.
+             */
+            Map<Class<?>, Integer> p = new HashMap<>();
+            {
+                p.put(SpoutNode.class, 0);
+                p.put(UpdateStateByKeyProcessor.class, 1);
+                p.put(ProcessorNode.class, 2);
+                p.put(StateQueryProcessor.class, 3);
+                p.put(PartitionNode.class, 4);
+                p.put(WindowNode.class, 5);
+                p.put(SinkNode.class, 6);
+            }
             @Override
             public int compare(Node n1, Node n2) {
-                return getPriority(n1.getClass()) - getPriority(n2.getClass());
+                return getPriority(n1) - getPriority(n2);
             }
 
-            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;
+            private int getPriority(Node node) {
+                Integer priority;
+                // check if processor has specific priority first
+                if (node instanceof ProcessorNode) {
+                    Processor processor = ((ProcessorNode) node).getProcessor();
+                    priority = p.get(processor.getClass());
+                    if (priority != null) {
+                        return priority;
                     }
                 }
+                priority = p.get(node.getClass());
+                if (priority != null) {
+                    return priority;
+                }
                 return Integer.MAX_VALUE;
             }
         });
@@ -319,7 +361,7 @@ public class StreamBuilder {
         }
     }
 
-    private Node parentNode(Node curNode) {
+    Node parentNode(Node curNode) {
         Set<Node> parentNode = parentNodes(curNode);
         if (parentNode.size() > 1) {
             throw new IllegalArgumentException("Node " + curNode + " has more than one parent node.");
@@ -350,7 +392,6 @@ public class StreamBuilder {
         String boltId = UniqueIdGen.getInstance().getUniqueBoltId();
         for (ProcessorNode processorNode : curGroup) {
             processorNode.setComponentId(boltId);
-            processorNode.setWindowed(isWindowed(processorNode));
             processorNode.setWindowedParentStreams(getWindowedParentStreams(processorNode));
         }
         final Set<ProcessorNode> initialProcessors = initialProcessors(curGroup);
@@ -397,12 +438,7 @@ public class StreamBuilder {
         Set<WindowNode> windowNodes = new HashSet<>();
         Set<Node> parents;
         for (ProcessorNode processorNode : initialProcessors) {
-            if (processorNode.getProcessor() instanceof JoinProcessor) {
-                String leftStream = ((JoinProcessor) processorNode.getProcessor()).getLeftStream();
-                parents = processorNode.getParents(leftStream);
-            } else {
-                parents = parentNodes(processorNode);
-            }
+            parents = parentNodes(processorNode);
             for (Node node : parents) {
                 if (windowInfo.containsKey(node)) {
                     windowNodes.add(windowInfo.get(node));
@@ -438,7 +474,7 @@ public class StreamBuilder {
         }
         for (Node parent : parentNodes(sinkNode)) {
             for (String stream : sinkNode.getParentStreams(parent)) {
-                declareStream(boltDeclarer, parent, stream, nodeGroupingInfo.get(parent, stream));
+                declareGrouping(boltDeclarer, parent, stream, nodeGroupingInfo.get(parent, stream));
             }
         }
     }
@@ -511,16 +547,8 @@ public class StreamBuilder {
     private Set<String> getWindowedParentStreams(ProcessorNode processorNode) {
         Set<String> 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<String>() {
-                        @Override
-                        public boolean apply(String input) {
-                            return !StreamUtil.isSinkStream(input);
-                        }
-                    }));
-                }
+            if (parent instanceof ProcessorNode && parent.isWindowed()) {
+                res.addAll(parent.getOutputStreams());
             }
         }
         return res;
@@ -539,10 +567,14 @@ public class StreamBuilder {
                     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));
+                        declareGrouping(boltDeclarer, parent, stream, nodeGroupingInfo.get(parent, stream));
                         // put global stream id for spouts
                         if (parent.getComponentId().startsWith("spout")) {
                             stream = parent.getComponentId() + stream;
+                        } else {
+                            // subscribe to parent's punctuation stream
+                            String punctuationStream = StreamUtil.getPunctuationStream(stream);
+                            declareGrouping(boltDeclarer, parent, punctuationStream, GroupingInfo.all());
                         }
                         streamToInitialProcessor.put(stream, curNode);
                     }
@@ -552,7 +584,7 @@ public class StreamBuilder {
         return streamToInitialProcessor;
     }
 
-    private void declareStream(BoltDeclarer boltDeclarer, Node parent, String streamId, GroupingInfo grouping) {
+    private void declareGrouping(BoltDeclarer boltDeclarer, Node parent, String streamId, GroupingInfo grouping) {
         if (grouping == null) {
             boltDeclarer.shuffleGrouping(parent.getComponentId(), streamId);
         } else {
@@ -572,20 +604,4 @@ public class StreamBuilder {
         }
         return nodes;
     }
-
-    private boolean isWindowed(Node curNode) {
-        for (Node parent : StreamUtil.<Node>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;
-    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/StreamUtil.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/StreamUtil.java b/storm-core/src/jvm/org/apache/storm/streams/StreamUtil.java
index 0531ff6..e0bf7d3 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/StreamUtil.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/StreamUtil.java
@@ -17,11 +17,14 @@
  */
 package org.apache.storm.streams;
 
+import org.apache.storm.tuple.Fields;
 import org.jgrapht.DirectedGraph;
 
 import java.util.ArrayList;
 import java.util.List;
 
+import static org.apache.storm.streams.WindowNode.PUNCTUATION;
+
 public class StreamUtil {
     @SuppressWarnings("unchecked")
     public static <T> List<T> getParents(DirectedGraph<Node, Edge> graph, Node node) {
@@ -43,20 +46,24 @@ public class StreamUtil {
         return ret;
     }
 
-
-    public static boolean isSinkStream(String streamId) {
-        return streamId.endsWith("__sink");
+    public static boolean isPunctuation(Object value) {
+        return PUNCTUATION.equals(value);
     }
 
-    public static String getSinkStream(String streamId) {
-        return streamId + "__sink";
+    public static String getPunctuationStream(String stream) {
+        return stream + PUNCTUATION;
     }
 
-    public static boolean isPunctuation(Object value) {
-        if (value instanceof Pair) {
-            value = ((Pair) value).getFirst();
+    public static String getSourceStream(String stream) {
+        int idx = stream.lastIndexOf(PUNCTUATION);
+        if (idx > 0) {
+            return stream.substring(0, idx);
         }
-        return WindowNode.PUNCTUATION.equals(value);
+        return stream;
+    }
+
+    public static Fields getPunctuationFields() {
+        return new Fields(PUNCTUATION);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/Tuple3.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/Tuple3.java b/storm-core/src/jvm/org/apache/storm/streams/Tuple3.java
deleted file mode 100644
index 77973f2..0000000
--- a/storm-core/src/jvm/org/apache/storm/streams/Tuple3.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- * 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 <T1> the type of the first element
- * @param <T2> the type of the second element
- * @param <T3> the type of the third element
- */
-public class Tuple3<T1, T2, T3> {
-    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 + ")";
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/WindowNode.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/WindowNode.java b/storm-core/src/jvm/org/apache/storm/streams/WindowNode.java
index a0e831a..c1783b5 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/WindowNode.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/WindowNode.java
@@ -29,6 +29,7 @@ public class WindowNode extends Node {
 
     WindowNode(Window<?, ?> windowParams, String outputStream, Fields outputFields) {
         super(outputStream, outputFields);
+        setWindowed(true);
         this.windowParams = windowParams;
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/operations/Aggregator.java
----------------------------------------------------------------------
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
deleted file mode 100644
index e3feaf4..0000000
--- a/storm-core/src/jvm/org/apache/storm/streams/operations/Aggregator.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * 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 <T> the original value type
- * @param <R> the aggregated value type
- */
-public interface Aggregator<T, R> 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);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/operations/BiFunction.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/BiFunction.java b/storm-core/src/jvm/org/apache/storm/streams/operations/BiFunction.java
new file mode 100644
index 0000000..7732e47
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/BiFunction.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;
+
+/**
+ * a function that accepts two arguments and produces a result.
+ *
+ * @param <T> the type of the first argument to the function
+ * @param <U> the type of the second argument to the function
+ * @param <R> the type of the result of the function
+ */
+public interface BiFunction<T, U, R> extends Operation {
+    /**
+     * Applies this function to the given arguments.
+     *
+     * @param input1 the first function argument
+     * @param input2 the second function argument
+     * @return the function result
+     */
+
+    R apply(T input1, U input2);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/operations/CombinerAggregator.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/CombinerAggregator.java b/storm-core/src/jvm/org/apache/storm/streams/operations/CombinerAggregator.java
new file mode 100644
index 0000000..a74b2c8
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/CombinerAggregator.java
@@ -0,0 +1,97 @@
+/**
+ * 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 <T> the original value type
+ * @param <A> the accumulator type
+ * @param <R> the result type
+ */
+public interface CombinerAggregator<T, A, R> extends Operation {
+    /**
+     * The initial value of the accumulator to start with.
+     *
+     * @return the initial value of the accumulator
+     */
+    A init();
+
+    /**
+     * Updates the accumulator by applying the current accumulator with the value.
+     *
+     * @param accumulator the current accumulator
+     * @param value       the value
+     * @return the updated accumulator
+     */
+    A apply(A accumulator, T value);
+
+    /**
+     * Merges two accumulators and returns the merged accumulator.
+     *
+     * @param accum1 the first accumulator
+     * @param accum2 the second accumulator
+     * @return the merged accumulator
+     */
+    A merge(A accum1, A accum2);
+
+    /**
+     * Produces a result value out of the accumulator.
+     *
+     * @param accum the accumulator
+     * @return the result
+     */
+    R result(A accum);
+
+    /**
+     * A static factory to create a {@link CombinerAggregator} based on initial value, accumulator and combiner.
+     *
+     * @param initialValue the initial value of the result to start with
+     * @param accumulator  a function that accumulates values into a partial result
+     * @param combiner     a function that combines partially accumulated results
+     * @param <T>          the value type
+     * @param <R>          the result type
+     * @return the {@link CombinerAggregator}
+     */
+    static <T, R> CombinerAggregator<T, R, R> of(R initialValue,
+                                                 BiFunction<? super R, ? super T, ? extends R> accumulator,
+                                                 BiFunction<? super R, ? super R, ? extends R> combiner) {
+        return new CombinerAggregator<T, R, R>() {
+            @Override
+            public R init() {
+                return initialValue;
+            }
+
+            @Override
+            public R apply(R aggregate, T value) {
+                return accumulator.apply(aggregate, value);
+            }
+
+            @Override
+            public R merge(R accum1, R accum2) {
+                return combiner.apply(accum1, accum2);
+            }
+
+            @Override
+            public R result(R accum) {
+                return accum;
+            }
+        };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/operations/Reducer.java
----------------------------------------------------------------------
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
index 04ee70d..48dc6fe 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/operations/Reducer.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/Reducer.java
@@ -23,7 +23,7 @@ package org.apache.storm.streams.operations;
  *
  * @param <T> the type of the arguments and the result
  */
-public interface Reducer<T> extends Operation {
+public interface Reducer<T> extends BiFunction<T, T, T> {
     /**
      * Applies this function to the given arguments.
      *

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/operations/StateUpdater.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/StateUpdater.java b/storm-core/src/jvm/org/apache/storm/streams/operations/StateUpdater.java
new file mode 100644
index 0000000..a1b1383
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/StateUpdater.java
@@ -0,0 +1,67 @@
+/**
+ * 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 updating state.
+ *
+ * @param <T> the value type
+ * @param <S> the state type
+ */
+public interface StateUpdater<T, S> extends Operation {
+    /**
+     * The initial value of the state to start with.
+     *
+     * @return the initial value of the state
+     */
+    S init();
+
+    /**
+     * Returns a new state by applying the value on the current state.
+     *
+     * @param state the current state
+     * @param value the value
+     * @return the new state
+     */
+    S apply(S state, T value);
+
+    /**
+     * A static factory to create a {@link StateUpdater} based on an initial value of the state and a
+     * state update function.
+     *
+     * @param initialValue  the intial value of the state
+     * @param stateUpdateFn the state update function
+     * @param <T>           the value type
+     * @param <S>           the state type
+     * @return the {@link StateUpdater}
+     */
+    static <T, S> StateUpdater<T, S> of(S initialValue,
+                                        BiFunction<? super S, ? super T, ? extends S> stateUpdateFn) {
+        return new StateUpdater<T, S>() {
+            @Override
+            public S init() {
+                return initialValue;
+            }
+
+            @Override
+            public S apply(S state, T value) {
+                return stateUpdateFn.apply(state, value);
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Count.java
----------------------------------------------------------------------
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
index fd02d2a..6589ed6 100644
--- 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
@@ -17,21 +17,31 @@
  */
 package org.apache.storm.streams.operations.aggregators;
 
-import org.apache.storm.streams.operations.Aggregator;
+import org.apache.storm.streams.operations.CombinerAggregator;
 
 /**
  * Computes the count of values.
  *
  * @param <T> the value type
  */
-public class Count<T> implements Aggregator<T, Long> {
+public class Count<T> implements CombinerAggregator<T, Long, Long> {
     @Override
     public Long init() {
         return 0L;
     }
 
     @Override
-    public Long apply(T value, Long aggregate) {
-        return aggregate + 1;
+    public Long apply(Long accum, T value) {
+        return accum + 1;
+    }
+
+    @Override
+    public Long merge(Long accum1, Long accum2) {
+        return accum1 + accum2;
+    }
+
+    @Override
+    public Long result(Long accum) {
+        return accum;
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Sum.java
----------------------------------------------------------------------
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
index e232075..df11d99 100644
--- 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
@@ -17,19 +17,29 @@
  */
 package org.apache.storm.streams.operations.aggregators;
 
-import org.apache.storm.streams.operations.Aggregator;
+import org.apache.storm.streams.operations.CombinerAggregator;
 
 /**
  * Computes the long sum of the input values
  */
-public class Sum implements Aggregator<Number, Long> {
+public class Sum implements CombinerAggregator<Number, Long, Long> {
     @Override
     public Long init() {
         return 0L;
     }
 
     @Override
-    public Long apply(Number value, Long aggregate) {
+    public Long apply(Long aggregate, Number value) {
         return value.longValue() + aggregate;
     }
+
+    @Override
+    public Long merge(Long accum1, Long accum2) {
+        return accum1 + accum2;
+    }
+
+    @Override
+    public Long result(Long accum) {
+        return accum;
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/TupleValueMappers.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/TupleValueMappers.java b/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/TupleValueMappers.java
new file mode 100644
index 0000000..f900f84
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/TupleValueMappers.java
@@ -0,0 +1,174 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.tuple.Tuple3;
+import org.apache.storm.streams.tuple.Tuple4;
+import org.apache.storm.streams.tuple.Tuple5;
+import org.apache.storm.streams.tuple.Tuple6;
+import org.apache.storm.streams.tuple.Tuple7;
+import org.apache.storm.streams.tuple.Tuple8;
+import org.apache.storm.streams.tuple.Tuple9;
+import org.apache.storm.streams.tuple.Tuple10;
+import org.apache.storm.tuple.Tuple;
+
+/**
+ * Factory for constructing typed tuples from a {@link Tuple}
+ * based on indicies.
+ */
+@SuppressWarnings("unchecked")
+public final class TupleValueMappers {
+    private TupleValueMappers() {
+    }
+
+    public static <T1, T2, T3>
+    TupleValueMapper<Tuple3<T1, T2, T3>> of(int index1,
+                                            int index2,
+                                            int index3) {
+        return input -> new Tuple3<>(
+                (T1) input.getValue(index1),
+                (T2) input.getValue(index2),
+                (T3) input.getValue(index3));
+    }
+
+    public static <T1, T2, T3, T4>
+    TupleValueMapper<Tuple4<T1, T2, T3, T4>> of(int index1,
+                                                int index2,
+                                                int index3,
+                                                int index4) {
+        return input -> new Tuple4<>(
+                (T1) input.getValue(index1),
+                (T2) input.getValue(index2),
+                (T3) input.getValue(index3),
+                (T4) input.getValue(index4));
+    }
+
+    public static <T1, T2, T3, T4, T5>
+    TupleValueMapper<Tuple5<T1, T2, T3, T4, T5>> of(int index1,
+                                                    int index2,
+                                                    int index3,
+                                                    int index4,
+                                                    int index5) {
+        return input -> new Tuple5<>(
+                (T1) input.getValue(index1),
+                (T2) input.getValue(index2),
+                (T3) input.getValue(index3),
+                (T4) input.getValue(index4),
+                (T5) input.getValue(index5));
+    }
+
+    public static <T1, T2, T3, T4, T5, T6>
+    TupleValueMapper<Tuple6<T1, T2, T3, T4, T5, T6>> of(int index1,
+                                                        int index2,
+                                                        int index3,
+                                                        int index4,
+                                                        int index5,
+                                                        int index6) {
+        return input -> new Tuple6<>(
+                (T1) input.getValue(index1),
+                (T2) input.getValue(index2),
+                (T3) input.getValue(index3),
+                (T4) input.getValue(index4),
+                (T5) input.getValue(index5),
+                (T6) input.getValue(index6));
+    }
+
+    public static <T1, T2, T3, T4, T5, T6, T7>
+    TupleValueMapper<Tuple7<T1, T2, T3, T4, T5, T6, T7>> of(int index1,
+                                                            int index2,
+                                                            int index3,
+                                                            int index4,
+                                                            int index5,
+                                                            int index6,
+                                                            int index7) {
+        return input -> new Tuple7<>(
+                (T1) input.getValue(index1),
+                (T2) input.getValue(index2),
+                (T3) input.getValue(index3),
+                (T4) input.getValue(index4),
+                (T5) input.getValue(index5),
+                (T6) input.getValue(index6),
+                (T7) input.getValue(index7));
+    }
+
+    public static <T1, T2, T3, T4, T5, T6, T7, T8>
+    TupleValueMapper<Tuple8<T1, T2, T3, T4, T5, T6, T7, T8>> of(int index1,
+                                                                int index2,
+                                                                int index3,
+                                                                int index4,
+                                                                int index5,
+                                                                int index6,
+                                                                int index7,
+                                                                int index8) {
+        return input -> new Tuple8<>(
+                (T1) input.getValue(index1),
+                (T2) input.getValue(index2),
+                (T3) input.getValue(index3),
+                (T4) input.getValue(index4),
+                (T5) input.getValue(index5),
+                (T6) input.getValue(index6),
+                (T7) input.getValue(index7),
+                (T8) input.getValue(index8));
+    }
+
+    public static <T1, T2, T3, T4, T5, T6, T7, T8, T9>
+    TupleValueMapper<Tuple9<T1, T2, T3, T4, T5, T6, T7, T8, T9>> of(int index1,
+                                                                    int index2,
+                                                                    int index3,
+                                                                    int index4,
+                                                                    int index5,
+                                                                    int index6,
+                                                                    int index7,
+                                                                    int index8,
+                                                                    int index9) {
+        return input -> new Tuple9<>(
+                (T1) input.getValue(index1),
+                (T2) input.getValue(index2),
+                (T3) input.getValue(index3),
+                (T4) input.getValue(index4),
+                (T5) input.getValue(index5),
+                (T6) input.getValue(index6),
+                (T7) input.getValue(index7),
+                (T8) input.getValue(index8),
+                (T9) input.getValue(index9));
+    }
+
+    public static <T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>
+    TupleValueMapper<Tuple10<T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>> of(int index1,
+                                                                          int index2,
+                                                                          int index3,
+                                                                          int index4,
+                                                                          int index5,
+                                                                          int index6,
+                                                                          int index7,
+                                                                          int index8,
+                                                                          int index9,
+                                                                          int index10) {
+        return input -> new Tuple10<>(
+                (T1) input.getValue(index1),
+                (T2) input.getValue(index2),
+                (T3) input.getValue(index3),
+                (T4) input.getValue(index4),
+                (T5) input.getValue(index5),
+                (T6) input.getValue(index6),
+                (T7) input.getValue(index7),
+                (T8) input.getValue(index8),
+                (T9) input.getValue(index9),
+                (T10) input.getValue(index10));
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java
----------------------------------------------------------------------
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
index d53a52d..c10113b 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java
@@ -18,37 +18,60 @@
 package org.apache.storm.streams.processors;
 
 import org.apache.storm.streams.Pair;
-import org.apache.storm.streams.operations.Aggregator;
+import org.apache.storm.streams.operations.CombinerAggregator;
 
 import java.util.HashMap;
 import java.util.Map;
 
-public class AggregateByKeyProcessor<K, V, R> extends BaseProcessor<Pair<K, V>> implements BatchProcessor {
-    private final Aggregator<V, R> aggregator;
-    private final Map<K, R> state = new HashMap<>();
+public class AggregateByKeyProcessor<K, V, A, R> extends BaseProcessor<Pair<K, V>> implements BatchProcessor {
+    private final CombinerAggregator<V, A, R> aggregator;
+    private final boolean emitAggregate;
+    private final Map<K, A> state = new HashMap<>();
 
-    public AggregateByKeyProcessor(Aggregator<V, R> aggregator) {
+    public AggregateByKeyProcessor(CombinerAggregator<V, A, R> aggregator) {
+        this(aggregator, false);
+    }
+
+    public AggregateByKeyProcessor(CombinerAggregator<V, A, R> aggregator, boolean emitAggregate) {
         this.aggregator = aggregator;
+        this.emitAggregate = emitAggregate;
     }
 
     @Override
     public void execute(Pair<K, V> input) {
         K key = input.getFirst();
         V val = input.getSecond();
-        R agg = state.get(key);
-        if (agg == null) {
-            agg = aggregator.init();
+        A accumulator = state.get(key);
+        if (accumulator == null) {
+            accumulator = aggregator.init();
+        }
+        state.put(key, aggregator.apply(accumulator, val));
+        if (emitAggregate) {
+            mayBeForwardAggUpdate(Pair.of(key, state.get(key)));
+        } else {
+            mayBeForwardAggUpdate(Pair.of(key, aggregator.result(state.get(key))));
         }
-        state.put(key, aggregator.apply(val, agg));
-        mayBeForwardAggUpdate(Pair.of(key, state.get(key)));
     }
 
     @Override
     public void finish() {
-        for (Map.Entry<K, R> entry : state.entrySet()) {
-            context.forward(Pair.of(entry.getKey(), entry.getValue()));
+        for (Map.Entry<K, A> entry : state.entrySet()) {
+            if (emitAggregate) {
+                context.forward(Pair.of(entry.getKey(), entry.getValue()));
+            } else {
+                context.forward(Pair.of(entry.getKey(), aggregator.result(entry.getValue())));
+            }
+
         }
         state.clear();
     }
 
+    @Override
+    public String toString() {
+        return "AggregateByKeyProcessor{" +
+                "aggregator=" + aggregator +
+                ", emitAggregate=" + emitAggregate +
+                ", state=" + state +
+                "}";
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java
----------------------------------------------------------------------
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
index c5a1906..d169345 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java
@@ -17,14 +17,20 @@
  */
 package org.apache.storm.streams.processors;
 
-import org.apache.storm.streams.operations.Aggregator;
+import org.apache.storm.streams.operations.CombinerAggregator;
 
-public class AggregateProcessor<T, R> extends BaseProcessor<T> implements BatchProcessor {
-    private final Aggregator<T, R> aggregator;
-    private R state;
+public class AggregateProcessor<T, A, R> extends BaseProcessor<T> implements BatchProcessor {
+    private final CombinerAggregator<T, A, R> aggregator;
+    private final boolean emitAggregate;
+    private A state;
 
-    public AggregateProcessor(Aggregator<T, R> aggregator) {
+    public AggregateProcessor(CombinerAggregator<T, A, R> aggregator) {
+        this(aggregator, false);
+    }
+
+    public AggregateProcessor(CombinerAggregator<T, A, R> aggregator, boolean emitAggregate) {
         this.aggregator = aggregator;
+        this.emitAggregate = emitAggregate;
     }
 
     @Override
@@ -32,14 +38,32 @@ public class AggregateProcessor<T, R> extends BaseProcessor<T> implements BatchP
         if (state == null) {
             state = aggregator.init();
         }
-        R curAggregate = (state != null) ? state : aggregator.init();
-        state = aggregator.apply(input, curAggregate);
-        mayBeForwardAggUpdate(state);
+        state = aggregator.apply(state, input);
+        if (emitAggregate) {
+            mayBeForwardAggUpdate(state);
+        } else {
+            mayBeForwardAggUpdate(aggregator.result(state));
+        }
     }
 
     @Override
     public void finish() {
-        context.forward(state);
-        state = null;
+        if (state != null) {
+            if (emitAggregate) {
+                context.forward(state);
+            } else {
+                context.forward(aggregator.result(state));
+            }
+            state = null;
+        }
+    }
+
+    @Override
+    public String toString() {
+        return "AggregateProcessor{" +
+                "aggregator=" + aggregator +
+                ", emitAggregate=" + emitAggregate +
+                ", state=" + state +
+                "}";
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/processors/BranchProcessor.java
----------------------------------------------------------------------
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
index f8bc739..421a5a7 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/processors/BranchProcessor.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/BranchProcessor.java
@@ -23,15 +23,15 @@ import java.util.HashMap;
 import java.util.Map;
 
 public class BranchProcessor<T> extends BaseProcessor<T> {
-    private final Map<Predicate<T>, String> predicateToStream = new HashMap<>();
+    private final Map<Predicate<? super T>, String> predicateToStream = new HashMap<>();
 
-    public void addPredicate(Predicate<T> predicate, String stream) {
+    public void addPredicate(Predicate<? super T> predicate, String stream) {
         predicateToStream.put(predicate, stream);
     }
 
     @Override
     public void execute(T input) {
-        for (Map.Entry<Predicate<T>, String> entry : predicateToStream.entrySet()) {
+        for (Map.Entry<Predicate<? super T>, String> entry : predicateToStream.entrySet()) {
             if (entry.getKey().test(input)) {
                 context.forward(input, entry.getValue());
                 break;

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/processors/EmittingProcessorContext.java
----------------------------------------------------------------------
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
index d841080..a85eaf7 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/processors/EmittingProcessorContext.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/EmittingProcessorContext.java
@@ -46,11 +46,11 @@ public class EmittingProcessorContext implements ProcessorContext {
     private static final Logger LOG = LoggerFactory.getLogger(EmittingProcessorContext.class);
     private final ProcessorNode processorNode;
     private final String outputStreamId;
+    private final String punctuationStreamId;
     private final OutputCollector collector;
     private final Fields outputFields;
     private final Values punctuation;
     private final List<RefCountedTuple> anchors = new ArrayList<>();
-    private boolean emitPunctuation = true;
     private long eventTimestamp;
     private String timestampField;
 
@@ -59,23 +59,20 @@ public class EmittingProcessorContext implements ProcessorContext {
         this.outputStreamId = outputStreamId;
         this.collector = collector;
         outputFields = processorNode.getOutputFields();
-        punctuation = createPunctuation();
+        punctuation = new Values(PUNCTUATION);
+        punctuationStreamId = StreamUtil.getPunctuationStream(outputStreamId);
     }
 
     @Override
     public <T> 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");
-            }
+        if (PUNCTUATION.equals(input)) {
+            emit(punctuation, punctuationStreamId);
             maybeAck();
+        } else if (processorNode.emitsPair()) {
+            Pair<?, ?> value = (Pair<?, ?>) input;
+            emit(new Values(value.getFirst(), value.getSecond()), outputStreamId);
         } else {
-            emit(new Values(input));
+            emit(new Values(input), outputStreamId);
         }
     }
 
@@ -96,10 +93,6 @@ public class EmittingProcessorContext implements ProcessorContext {
         return processorNode.getWindowedParentStreams();
     }
 
-    public void setEmitPunctuation(boolean emitPunctuation) {
-        this.emitPunctuation = emitPunctuation;
-    }
-
     public void setTimestampField(String fieldName) {
         timestampField = fieldName;
     }
@@ -128,14 +121,6 @@ public class EmittingProcessorContext implements ProcessorContext {
         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) {
@@ -154,7 +139,7 @@ public class EmittingProcessorContext implements ProcessorContext {
         return anchors.stream().map(RefCountedTuple::tuple).collect(Collectors.toList());
     }
 
-    private void emit(Values values) {
+    private void emit(Values values, String outputStreamId) {
         if (timestampField != null) {
             values.add(eventTimestamp);
         }

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/processors/JoinProcessor.java
----------------------------------------------------------------------
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
index d56cfea..05cad8d 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/processors/JoinProcessor.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/JoinProcessor.java
@@ -20,12 +20,14 @@ 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 org.apache.storm.streams.tuple.Tuple3;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 
 /**
  * Provides equi-join implementation based on simple hash-join.
@@ -36,11 +38,25 @@ public class JoinProcessor<K, R, V1, V2> extends BaseProcessor<Pair<K, ?>> imple
     private final String rightStream;
     private final List<Pair<K, V1>> leftRows = new ArrayList<>();
     private final List<Pair<K, V2>> rightRows = new ArrayList<>();
+    private final JoinType leftType;
+    private final JoinType rightType;
+
+    public enum JoinType {
+        INNER,
+        OUTER
+    }
 
     public JoinProcessor(String leftStream, String rightStream, ValueJoiner<V1, V2, R> valueJoiner) {
+        this(leftStream, rightStream, valueJoiner, JoinType.INNER, JoinType.INNER);
+    }
+
+    public JoinProcessor(String leftStream, String rightStream, ValueJoiner<V1, V2, R> valueJoiner,
+                         JoinType leftType, JoinType rightType) {
         this.valueJoiner = valueJoiner;
         this.leftStream = leftStream;
         this.rightStream = rightStream;
+        this.leftType = leftType;
+        this.rightType = rightType;
     }
 
     @Override
@@ -78,30 +94,54 @@ public class JoinProcessor<K, R, V1, V2> extends BaseProcessor<Pair<K, ?>> imple
         return rightStream;
     }
 
+    /*
+     * performs a hash-join by constructing a hash map of the smaller set, iterating over the
+     * larger set and finding matching rows in the hash map.
+     */
     private void joinAndForward(List<Pair<K, V1>> leftRows, List<Pair<K, V2>> rightRows) {
-        if (leftRows.size() <= rightRows.size()) {
-            for (Tuple3<K, V1, V2> res : join(getJoinTable(leftRows), rightRows)) {
+        if (leftRows.size() < rightRows.size()) {
+            for (Tuple3<K, V1, V2> res : join(getJoinTable(leftRows), rightRows, leftType, rightType)) {
                 context.forward(Pair.of(res._1, valueJoiner.apply(res._2, res._3)));
             }
         } else {
-            for (Tuple3<K, V2, V1> res : join(getJoinTable(rightRows), leftRows)) {
+            for (Tuple3<K, V2, V1> res : join(getJoinTable(rightRows), leftRows, rightType, leftType)) {
                 context.forward(Pair.of(res._1, valueJoiner.apply(res._3, res._2)));
             }
         }
     }
 
-    private <T1, T2> List<Tuple3<K, T1, T2>> join(Multimap<K, T1> tab, List<Pair<K, T2>> rows) {
+    /*
+     * returns list of Tuple3 (key, val from table, val from row)
+     */
+    private <T1, T2> List<Tuple3<K, T1, T2>> join(Multimap<K, T1> tab, List<Pair<K, T2>> rows,
+                                                  JoinType leftType, JoinType rightType) {
         List<Tuple3<K, T1, T2>> res = new ArrayList<>();
         for (Pair<K, T2> row : rows) {
-            for (T1 mapValue : tab.get(row.getFirst())) {
-                if (mapValue != null) {
+            K key = row.getFirst();
+            Collection<T1> values = tab.removeAll(key);
+            if (values.isEmpty()) {
+                if (rightType == JoinType.OUTER) {
+                    res.add(new Tuple3<>(row.getFirst(), null, row.getSecond()));
+                }
+            } else {
+                for (T1 mapValue : values) {
                     res.add(new Tuple3<>(row.getFirst(), mapValue, row.getSecond()));
                 }
             }
         }
+        // whatever remains in the tab are non matching left rows.
+        if (leftType == JoinType.OUTER) {
+            for (Map.Entry<K, T1> row : tab.entries()) {
+                res.add(new Tuple3<>(row.getKey(), row.getValue(), null));
+            }
+        }
         return res;
     }
 
+    /*
+     * key1 -> (val1, val2 ..)
+     * key2 -> (val3, val4 ..)
+     */
     private <T> Multimap<K, T> getJoinTable(List<Pair<K, T>> rows) {
         Multimap<K, T> m = ArrayListMultimap.create();
         for (Pair<K, T> v : rows) {

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateByKeyProcessor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateByKeyProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateByKeyProcessor.java
new file mode 100644
index 0000000..57ad845
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateByKeyProcessor.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.CombinerAggregator;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class MergeAggregateByKeyProcessor<K, V, A, R> extends BaseProcessor<Pair<K, A>> implements BatchProcessor {
+    protected final CombinerAggregator<V, A, R> aggregator;
+    protected final Map<K, A> state = new HashMap<>();
+
+    public MergeAggregateByKeyProcessor(CombinerAggregator<V, A, R> aggregator) {
+        this.aggregator = aggregator;
+    }
+
+    @Override
+    public void execute(Pair<K, A> input) {
+        K key = input.getFirst();
+        A val = input.getSecond();
+        A accumulator = state.get(key);
+        if (accumulator == null) {
+            accumulator = aggregator.init();
+        }
+        state.put(key, aggregator.merge(accumulator, val));
+        mayBeForwardAggUpdate(Pair.of(key, aggregator.result(state.get(key))));
+    }
+
+    @Override
+    public void finish() {
+        for (Map.Entry<K, A> entry : state.entrySet()) {
+            context.forward(Pair.of(entry.getKey(), aggregator.result(entry.getValue())));
+        }
+        state.clear();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateProcessor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateProcessor.java
new file mode 100644
index 0000000..61b555b
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateProcessor.java
@@ -0,0 +1,47 @@
+/**
+ * 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.CombinerAggregator;
+
+public class MergeAggregateProcessor<T, A, R> extends BaseProcessor<A> implements BatchProcessor {
+    private final CombinerAggregator<T, A, R> aggregator;
+    private A state;
+
+    public MergeAggregateProcessor(CombinerAggregator<T, A, R> aggregator) {
+        this.aggregator = aggregator;
+    }
+
+    @Override
+    protected void execute(A input) {
+        if (state == null) {
+            state = aggregator.init();
+        }
+        state = aggregator.merge(state, input);
+        mayBeForwardAggUpdate(aggregator.result(state));
+    }
+
+    @Override
+    public void finish() {
+        if (state != null) {
+            context.forward(aggregator.result(state));
+            state = null;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java
index d64e114..0b90fb9 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java
@@ -35,7 +35,9 @@ public class ReduceProcessor<T> extends BaseProcessor<T> implements BatchProcess
 
     @Override
     public void finish() {
-        context.forward(agg);
-        agg = null;
+        if (agg != null) {
+            context.forward(agg);
+            agg = null;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/processors/UpdateStateByKeyProcessor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/UpdateStateByKeyProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/UpdateStateByKeyProcessor.java
index 9e068a0..5fb2730 100644
--- a/storm-core/src/jvm/org/apache/storm/streams/processors/UpdateStateByKeyProcessor.java
+++ b/storm-core/src/jvm/org/apache/storm/streams/processors/UpdateStateByKeyProcessor.java
@@ -19,14 +19,14 @@ package org.apache.storm.streams.processors;
 
 import org.apache.storm.state.KeyValueState;
 import org.apache.storm.streams.Pair;
-import org.apache.storm.streams.operations.Aggregator;
+import org.apache.storm.streams.operations.StateUpdater;
 
 public class UpdateStateByKeyProcessor<K, V, R> extends BaseProcessor<Pair<K, V>> implements StatefulProcessor<K, R> {
-    private final Aggregator<V, R> aggregator;
+    private final StateUpdater<V, R> stateUpdater;
     private KeyValueState<K, R> keyValueState;
 
-    public UpdateStateByKeyProcessor(Aggregator<V, R> aggregator) {
-        this.aggregator = aggregator;
+    public UpdateStateByKeyProcessor(StateUpdater<V, R> stateUpdater) {
+        this.stateUpdater = stateUpdater;
     }
 
     @Override
@@ -40,9 +40,9 @@ public class UpdateStateByKeyProcessor<K, V, R> extends BaseProcessor<Pair<K, V>
         V val = input.getSecond();
         R agg = keyValueState.get(key);
         if (agg == null) {
-            agg = aggregator.init();
+            agg = stateUpdater.init();
         }
-        R newAgg = aggregator.apply(val, agg);
+        R newAgg = stateUpdater.apply(agg, val);
         keyValueState.put(key, newAgg);
         context.forward(Pair.of(key, newAgg));
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple10.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple10.java b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple10.java
new file mode 100644
index 0000000..879d71c
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple10.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.tuple;
+
+/**
+ * A tuple of ten elements along the lines of Scala's Tuple.
+ *
+ * @param <T1>  the type of the first element
+ * @param <T2>  the type of the second element
+ * @param <T3>  the type of the third element
+ * @param <T4>  the type of the fourth element
+ * @param <T5>  the type of the fifth element
+ * @param <T6>  the type of the sixth element
+ * @param <T7>  the type of the seventh element
+ * @param <T8>  the type of the eighth element
+ * @param <T9>  the type of the ninth element
+ * @param <T10> the type of the tenth element
+ */
+public class Tuple10<T1, T2, T3, T4, T5, T6, T7, T8, T9, T10> {
+    public final T1 _1;
+    public final T2 _2;
+    public final T3 _3;
+    public final T4 _4;
+    public final T5 _5;
+    public final T6 _6;
+    public final T7 _7;
+    public final T8 _8;
+    public final T9 _9;
+    public final T10 _10;
+
+    /**
+     * Constructs a new tuple.
+     *
+     * @param _1  the first element
+     * @param _2  the second element
+     * @param _3  the third element
+     * @param _4  the fourth element
+     * @param _5  the fifth element
+     * @param _6  the sixth element
+     * @param _7  the seventh element
+     * @param _8  the eighth element
+     * @param _9  the ninth element
+     * @param _10 the tenth element
+     */
+    public Tuple10(T1 _1, T2 _2, T3 _3, T4 _4, T5 _5, T6 _6, T7 _7, T8 _8, T9 _9, T10 _10) {
+        this._1 = _1;
+        this._2 = _2;
+        this._3 = _3;
+        this._4 = _4;
+        this._5 = _5;
+        this._6 = _6;
+        this._7 = _7;
+        this._8 = _8;
+        this._9 = _9;
+        this._10 = _10;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        Tuple10<?, ?, ?, ?, ?, ?, ?, ?, ?, ?> tuple10 = (Tuple10<?, ?, ?, ?, ?, ?, ?, ?, ?, ?>) o;
+
+        if (_1 != null ? !_1.equals(tuple10._1) : tuple10._1 != null) return false;
+        if (_2 != null ? !_2.equals(tuple10._2) : tuple10._2 != null) return false;
+        if (_3 != null ? !_3.equals(tuple10._3) : tuple10._3 != null) return false;
+        if (_4 != null ? !_4.equals(tuple10._4) : tuple10._4 != null) return false;
+        if (_5 != null ? !_5.equals(tuple10._5) : tuple10._5 != null) return false;
+        if (_6 != null ? !_6.equals(tuple10._6) : tuple10._6 != null) return false;
+        if (_7 != null ? !_7.equals(tuple10._7) : tuple10._7 != null) return false;
+        if (_8 != null ? !_8.equals(tuple10._8) : tuple10._8 != null) return false;
+        if (_9 != null ? !_9.equals(tuple10._9) : tuple10._9 != null) return false;
+        return _10 != null ? _10.equals(tuple10._10) : tuple10._10 == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = _1 != null ? _1.hashCode() : 0;
+        result = 31 * result + (_2 != null ? _2.hashCode() : 0);
+        result = 31 * result + (_3 != null ? _3.hashCode() : 0);
+        result = 31 * result + (_4 != null ? _4.hashCode() : 0);
+        result = 31 * result + (_5 != null ? _5.hashCode() : 0);
+        result = 31 * result + (_6 != null ? _6.hashCode() : 0);
+        result = 31 * result + (_7 != null ? _7.hashCode() : 0);
+        result = 31 * result + (_8 != null ? _8.hashCode() : 0);
+        result = 31 * result + (_9 != null ? _9.hashCode() : 0);
+        result = 31 * result + (_10 != null ? _10.hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "(" + _1 + "," + _2 + "," + _3 + "," + _4 + "," + _5 + "," + _6 + "," + _7 + "," + _8 + "," + _9 + "," + _10 + ")";
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple3.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple3.java b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple3.java
new file mode 100644
index 0000000..514e169
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple3.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.tuple;
+
+/**
+ * A tuple of three elements along the lines of Scala's Tuple.
+ *
+ * @param <T1> the type of the first element
+ * @param <T2> the type of the second element
+ * @param <T3> the type of the third element
+ */
+public final class Tuple3<T1, T2, T3> {
+    public final T1 _1;
+    public final T2 _2;
+    public final T3 _3;
+
+    /**
+     * Constructs a new tuple.
+     *
+     * @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 boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        Tuple3<?, ?, ?> tuple3 = (Tuple3<?, ?, ?>) o;
+
+        if (_1 != null ? !_1.equals(tuple3._1) : tuple3._1 != null) return false;
+        if (_2 != null ? !_2.equals(tuple3._2) : tuple3._2 != null) return false;
+        return _3 != null ? _3.equals(tuple3._3) : tuple3._3 == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = _1 != null ? _1.hashCode() : 0;
+        result = 31 * result + (_2 != null ? _2.hashCode() : 0);
+        result = 31 * result + (_3 != null ? _3.hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "(" + _1 + "," + _2 + "," + _3 + ")";
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple4.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple4.java b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple4.java
new file mode 100644
index 0000000..5ae3fb1
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple4.java
@@ -0,0 +1,76 @@
+/**
+ * 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.tuple;
+
+/**
+ * A tuple of four elements along the lines of Scala's Tuple.
+ *
+ * @param <T1> the type of the first element
+ * @param <T2> the type of the second element
+ * @param <T3> the type of the third element
+ * @param <T4> the type of the fourth element
+ */
+public final class Tuple4<T1, T2, T3, T4> {
+    public final T1 _1;
+    public final T2 _2;
+    public final T3 _3;
+    public final T4 _4;
+
+    /**
+     * Constructs a new tuple.
+     *
+     * @param _1 the first element
+     * @param _2 the second element
+     * @param _3 the third element
+     * @param _4 the fourth element
+     */
+    public Tuple4(T1 _1, T2 _2, T3 _3, T4 _4) {
+        this._1 = _1;
+        this._2 = _2;
+        this._3 = _3;
+        this._4 = _4;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        Tuple4<?, ?, ?, ?> tuple4 = (Tuple4<?, ?, ?, ?>) o;
+
+        if (_1 != null ? !_1.equals(tuple4._1) : tuple4._1 != null) return false;
+        if (_2 != null ? !_2.equals(tuple4._2) : tuple4._2 != null) return false;
+        if (_3 != null ? !_3.equals(tuple4._3) : tuple4._3 != null) return false;
+        return _4 != null ? _4.equals(tuple4._4) : tuple4._4 == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = _1 != null ? _1.hashCode() : 0;
+        result = 31 * result + (_2 != null ? _2.hashCode() : 0);
+        result = 31 * result + (_3 != null ? _3.hashCode() : 0);
+        result = 31 * result + (_4 != null ? _4.hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "(" + _1 + "," + _2 + "," + _3 + "," + _4 + ")";
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple5.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple5.java b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple5.java
new file mode 100644
index 0000000..6b0f81e
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple5.java
@@ -0,0 +1,82 @@
+/**
+ * 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.tuple;
+
+/**
+ * A tuple of five elements along the lines of Scala's Tuple.
+ *
+ * @param <T1> the type of the first element
+ * @param <T2> the type of the second element
+ * @param <T3> the type of the third element
+ * @param <T4> the type of the fourth element
+ * @param <T5> the type of the fifth element
+ */
+public class Tuple5<T1, T2, T3, T4, T5> {
+    public final T1 _1;
+    public final T2 _2;
+    public final T3 _3;
+    public final T4 _4;
+    public final T5 _5;
+
+    /**
+     * Constructs a new tuple.
+     *
+     * @param _1 the first element
+     * @param _2 the second element
+     * @param _3 the third element
+     * @param _4 the fourth element
+     * @param _5 the fifth element
+     */
+    public Tuple5(T1 _1, T2 _2, T3 _3, T4 _4, T5 _5) {
+        this._1 = _1;
+        this._2 = _2;
+        this._3 = _3;
+        this._4 = _4;
+        this._5 = _5;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        Tuple5<?, ?, ?, ?, ?> tuple5 = (Tuple5<?, ?, ?, ?, ?>) o;
+
+        if (_1 != null ? !_1.equals(tuple5._1) : tuple5._1 != null) return false;
+        if (_2 != null ? !_2.equals(tuple5._2) : tuple5._2 != null) return false;
+        if (_3 != null ? !_3.equals(tuple5._3) : tuple5._3 != null) return false;
+        if (_4 != null ? !_4.equals(tuple5._4) : tuple5._4 != null) return false;
+        return _5 != null ? _5.equals(tuple5._5) : tuple5._5 == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = _1 != null ? _1.hashCode() : 0;
+        result = 31 * result + (_2 != null ? _2.hashCode() : 0);
+        result = 31 * result + (_3 != null ? _3.hashCode() : 0);
+        result = 31 * result + (_4 != null ? _4.hashCode() : 0);
+        result = 31 * result + (_5 != null ? _5.hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "(" + _1 + "," + _2 + "," + _3 + "," + _4 + "," + _5 + ")";
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple6.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple6.java b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple6.java
new file mode 100644
index 0000000..4c35e27
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple6.java
@@ -0,0 +1,89 @@
+/**
+ * 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.tuple;
+
+
+/**
+ * A tuple of six elements along the lines of Scala's Tuple.
+ *
+ * @param <T1> the type of the first element
+ * @param <T2> the type of the second element
+ * @param <T3> the type of the third element
+ * @param <T4> the type of the fourth element
+ * @param <T5> the type of the fifth element
+ * @param <T6> the type of the sixth element
+ */
+public class Tuple6<T1, T2, T3, T4, T5, T6> {
+    public final T1 _1;
+    public final T2 _2;
+    public final T3 _3;
+    public final T4 _4;
+    public final T5 _5;
+    public final T6 _6;
+
+    /**
+     * Constructs a new tuple.
+     *
+     * @param _1 the first element
+     * @param _2 the second element
+     * @param _3 the third element
+     * @param _4 the fourth element
+     * @param _5 the fifth element
+     * @param _6 the sixth element
+     */
+    public Tuple6(T1 _1, T2 _2, T3 _3, T4 _4, T5 _5, T6 _6) {
+        this._1 = _1;
+        this._2 = _2;
+        this._3 = _3;
+        this._4 = _4;
+        this._5 = _5;
+        this._6 = _6;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        Tuple6<?, ?, ?, ?, ?, ?> tuple6 = (Tuple6<?, ?, ?, ?, ?, ?>) o;
+
+        if (_1 != null ? !_1.equals(tuple6._1) : tuple6._1 != null) return false;
+        if (_2 != null ? !_2.equals(tuple6._2) : tuple6._2 != null) return false;
+        if (_3 != null ? !_3.equals(tuple6._3) : tuple6._3 != null) return false;
+        if (_4 != null ? !_4.equals(tuple6._4) : tuple6._4 != null) return false;
+        if (_5 != null ? !_5.equals(tuple6._5) : tuple6._5 != null) return false;
+        return _6 != null ? _6.equals(tuple6._6) : tuple6._6 == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = _1 != null ? _1.hashCode() : 0;
+        result = 31 * result + (_2 != null ? _2.hashCode() : 0);
+        result = 31 * result + (_3 != null ? _3.hashCode() : 0);
+        result = 31 * result + (_4 != null ? _4.hashCode() : 0);
+        result = 31 * result + (_5 != null ? _5.hashCode() : 0);
+        result = 31 * result + (_6 != null ? _6.hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "(" + _1 + "," + _2 + "," + _3 + "," + _4 + "," + _5 + "," + _6 + ")";
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple7.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple7.java b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple7.java
new file mode 100644
index 0000000..366e8e9
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple7.java
@@ -0,0 +1,94 @@
+/**
+ * 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.tuple;
+
+/**
+ * A tuple of seven elements along the lines of Scala's Tuple.
+ *
+ * @param <T1> the type of the first element
+ * @param <T2> the type of the second element
+ * @param <T3> the type of the third element
+ * @param <T4> the type of the fourth element
+ * @param <T5> the type of the fifth element
+ * @param <T6> the type of the sixth element
+ * @param <T7> the type of the seventh element
+ */
+public class Tuple7<T1, T2, T3, T4, T5, T6, T7> {
+    public final T1 _1;
+    public final T2 _2;
+    public final T3 _3;
+    public final T4 _4;
+    public final T5 _5;
+    public final T6 _6;
+    public final T7 _7;
+
+    /**
+     * Constructs a new tuple.
+     *
+     * @param _1 the first element
+     * @param _2 the second element
+     * @param _3 the third element
+     * @param _4 the fourth element
+     * @param _5 the fifth element
+     * @param _6 the sixth element
+     * @param _7 the seventh element
+     */
+    public Tuple7(T1 _1, T2 _2, T3 _3, T4 _4, T5 _5, T6 _6, T7 _7) {
+        this._1 = _1;
+        this._2 = _2;
+        this._3 = _3;
+        this._4 = _4;
+        this._5 = _5;
+        this._6 = _6;
+        this._7 = _7;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        Tuple7<?, ?, ?, ?, ?, ?, ?> tuple7 = (Tuple7<?, ?, ?, ?, ?, ?, ?>) o;
+
+        if (_1 != null ? !_1.equals(tuple7._1) : tuple7._1 != null) return false;
+        if (_2 != null ? !_2.equals(tuple7._2) : tuple7._2 != null) return false;
+        if (_3 != null ? !_3.equals(tuple7._3) : tuple7._3 != null) return false;
+        if (_4 != null ? !_4.equals(tuple7._4) : tuple7._4 != null) return false;
+        if (_5 != null ? !_5.equals(tuple7._5) : tuple7._5 != null) return false;
+        if (_6 != null ? !_6.equals(tuple7._6) : tuple7._6 != null) return false;
+        return _7 != null ? _7.equals(tuple7._7) : tuple7._7 == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = _1 != null ? _1.hashCode() : 0;
+        result = 31 * result + (_2 != null ? _2.hashCode() : 0);
+        result = 31 * result + (_3 != null ? _3.hashCode() : 0);
+        result = 31 * result + (_4 != null ? _4.hashCode() : 0);
+        result = 31 * result + (_5 != null ? _5.hashCode() : 0);
+        result = 31 * result + (_6 != null ? _6.hashCode() : 0);
+        result = 31 * result + (_7 != null ? _7.hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "(" + _1 + "," + _2 + "," + _3 + "," + _4 + "," + _5 + "," + _6 + "," + _7 + ")";
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/3a10865c/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple8.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple8.java b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple8.java
new file mode 100644
index 0000000..bf088df
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple8.java
@@ -0,0 +1,100 @@
+/**
+ * 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.tuple;
+
+/**
+ * A tuple of eight elements along the lines of Scala's Tuple.
+ *
+ * @param <T1> the type of the first element
+ * @param <T2> the type of the second element
+ * @param <T3> the type of the third element
+ * @param <T4> the type of the fourth element
+ * @param <T5> the type of the fifth element
+ * @param <T6> the type of the sixth element
+ * @param <T7> the type of the seventh element
+ * @param <T8> the type of the eighth element
+ */
+public class Tuple8<T1, T2, T3, T4, T5, T6, T7, T8> {
+    public final T1 _1;
+    public final T2 _2;
+    public final T3 _3;
+    public final T4 _4;
+    public final T5 _5;
+    public final T6 _6;
+    public final T7 _7;
+    public final T8 _8;
+
+    /**
+     * Constructs a new tuple.
+     *
+     * @param _1 the first element
+     * @param _2 the second element
+     * @param _3 the third element
+     * @param _4 the fourth element
+     * @param _5 the fifth element
+     * @param _6 the sixth element
+     * @param _7 the seventh element
+     * @param _8 the eighth element
+     */
+    public Tuple8(T1 _1, T2 _2, T3 _3, T4 _4, T5 _5, T6 _6, T7 _7, T8 _8) {
+        this._1 = _1;
+        this._2 = _2;
+        this._3 = _3;
+        this._4 = _4;
+        this._5 = _5;
+        this._6 = _6;
+        this._7 = _7;
+        this._8 = _8;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        Tuple8<?, ?, ?, ?, ?, ?, ?, ?> tuple8 = (Tuple8<?, ?, ?, ?, ?, ?, ?, ?>) o;
+
+        if (_1 != null ? !_1.equals(tuple8._1) : tuple8._1 != null) return false;
+        if (_2 != null ? !_2.equals(tuple8._2) : tuple8._2 != null) return false;
+        if (_3 != null ? !_3.equals(tuple8._3) : tuple8._3 != null) return false;
+        if (_4 != null ? !_4.equals(tuple8._4) : tuple8._4 != null) return false;
+        if (_5 != null ? !_5.equals(tuple8._5) : tuple8._5 != null) return false;
+        if (_6 != null ? !_6.equals(tuple8._6) : tuple8._6 != null) return false;
+        if (_7 != null ? !_7.equals(tuple8._7) : tuple8._7 != null) return false;
+        return _8 != null ? _8.equals(tuple8._8) : tuple8._8 == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = _1 != null ? _1.hashCode() : 0;
+        result = 31 * result + (_2 != null ? _2.hashCode() : 0);
+        result = 31 * result + (_3 != null ? _3.hashCode() : 0);
+        result = 31 * result + (_4 != null ? _4.hashCode() : 0);
+        result = 31 * result + (_5 != null ? _5.hashCode() : 0);
+        result = 31 * result + (_6 != null ? _6.hashCode() : 0);
+        result = 31 * result + (_7 != null ? _7.hashCode() : 0);
+        result = 31 * result + (_8 != null ? _8.hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "(" + _1 + "," + _2 + "," + _3 + "," + _4 + "," + _5 + "," + _6 + "," + _7 + "," + _8 + ")";
+    }
+}