You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by ni...@apache.org on 2016/12/01 22:51:01 UTC

[1/6] samza git commit: SAMZA-1054: Refactor Operator APIs

Repository: samza
Updated Branches:
  refs/heads/master a980c9622 -> 00543804b


http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java b/samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java
deleted file mode 100644
index f6b3ff8..0000000
--- a/samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java
+++ /dev/null
@@ -1,80 +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.samza.task;
-
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.MessageStreams.SystemMessageStream;
-import org.apache.samza.operators.data.IncomingSystemMessage;
-import org.apache.samza.operators.data.Offset;
-import org.apache.samza.operators.task.StreamOperatorTask;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-
-/**
- * Example implementation of unique key-based stream-stream join tasks
- *
- */
-public class JoinOperatorTask implements StreamOperatorTask {
-  class MessageType {
-    String joinKey;
-    List<String> joinFields = new ArrayList<>();
-  }
-
-  class JsonMessage extends InputJsonSystemMessage<MessageType> {
-
-    JsonMessage(String key, MessageType data, Offset offset, long timestamp, SystemStreamPartition partition) {
-      super(key, data, offset, timestamp, partition);
-    }
-  }
-
-  MessageStream<JsonMessage> joinOutput = null;
-
-  @Override public void initOperators(Collection<SystemMessageStream> sources) {
-    sources.forEach(source -> {
-        MessageStream<JsonMessage> newSource = source.map(this::getInputMessage);
-        if (joinOutput == null) {
-          joinOutput = newSource;
-        } else {
-          joinOutput = joinOutput.join(newSource, (m1, m2) -> this.myJoinResult(m1, m2));
-        }
-      });
-  }
-
-  private JsonMessage getInputMessage(IncomingSystemMessage ism) {
-    return new JsonMessage(
-        ((MessageType) ism.getMessage()).joinKey,
-        (MessageType) ism.getMessage(),
-        ism.getOffset(),
-        ism.getTimestamp(),
-        ism.getSystemStreamPartition());
-  }
-
-  JsonMessage myJoinResult(JsonMessage m1, JsonMessage m2) {
-    MessageType newJoinMsg = new MessageType();
-    newJoinMsg.joinKey = m1.getKey();
-    newJoinMsg.joinFields.addAll(m1.getMessage().joinFields);
-    newJoinMsg.joinFields.addAll(m2.getMessage().joinFields);
-    return new JsonMessage(m1.getMessage().joinKey, newJoinMsg, null, m1.getTimestamp(), null);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorAdaptorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorAdaptorTask.java b/samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorAdaptorTask.java
deleted file mode 100644
index 47d6b3a..0000000
--- a/samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorAdaptorTask.java
+++ /dev/null
@@ -1,80 +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.samza.task;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.impl.ChainedOperators;
-import org.apache.samza.operators.task.StreamOperatorTask;
-import org.apache.samza.system.SystemStreamPartition;
-import org.apache.samza.Partition;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-import java.lang.reflect.Field;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.*;
-
-
-public class TestStreamOperatorAdaptorTask {
-  Field userTaskField = null;
-  Field chainedOpsField = null;
-
-  @Before public void prep() throws NoSuchFieldException {
-    userTaskField = StreamOperatorAdaptorTask.class.getDeclaredField("userTask");
-    chainedOpsField = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
-    userTaskField.setAccessible(true);
-    chainedOpsField.setAccessible(true);
-  }
-
-
-  @Test public void testConstructor() throws IllegalAccessException {
-    StreamOperatorTask userTask = mock(StreamOperatorTask.class);
-    StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(userTask);
-    StreamOperatorTask taskMemberVar = (StreamOperatorTask) userTaskField.get(adaptorTask);
-    Map<SystemStreamPartition, ChainedOperators> chainsMap = (Map<SystemStreamPartition, ChainedOperators>) chainedOpsField.get(adaptorTask);
-    assertEquals(taskMemberVar, userTask);
-    assertTrue(chainsMap.isEmpty());
-  }
-
-  @Test public void testInit() throws Exception {
-    StreamOperatorTask userTask = mock(StreamOperatorTask.class);
-    StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(userTask);
-    Config mockConfig = mock(Config.class);
-    TaskContext mockContext = mock(TaskContext.class);
-    Set<SystemStreamPartition> testInputs = new HashSet() { {
-        this.add(new SystemStreamPartition("test-sys", "test-strm", new Partition(0)));
-        this.add(new SystemStreamPartition("test-sys", "test-strm", new Partition(1)));
-      } };
-    when(mockContext.getSystemStreamPartitions()).thenReturn(testInputs);
-    adaptorTask.init(mockConfig, mockContext);
-    verify(userTask, times(1)).initOperators(Mockito.anyCollection());
-    Map<SystemStreamPartition, ChainedOperators> chainsMap = (Map<SystemStreamPartition, ChainedOperators>) chainedOpsField.get(adaptorTask);
-    assertTrue(chainsMap.size() == 2);
-    assertTrue(chainsMap.containsKey(testInputs.toArray()[0]));
-    assertTrue(chainsMap.containsKey(testInputs.toArray()[1]));
-  }
-
-  // TODO: window and process methods to be added after implementation of ChainedOperators.create()
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java b/samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java
deleted file mode 100644
index 44efa6d..0000000
--- a/samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java
+++ /dev/null
@@ -1,105 +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.samza.task;
-
-import org.apache.samza.Partition;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.impl.ChainedOperators;
-import org.apache.samza.system.SystemStreamPartition;
-import org.junit.Test;
-
-import java.lang.reflect.Field;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-
-/**
- * Unit test for {@link org.apache.samza.operators.task.StreamOperatorTask}
- */
-public class TestStreamOperatorTasks {
-
-  private final WindowOperatorTask userTask = new WindowOperatorTask();
-
-  private final BroadcastOperatorTask splitTask = new BroadcastOperatorTask();
-
-  private final JoinOperatorTask joinTask = new JoinOperatorTask();
-
-  private final Set<SystemStreamPartition> inputPartitions = new HashSet<SystemStreamPartition>() { {
-      for (int i = 0; i < 4; i++) {
-        this.add(new SystemStreamPartition("my-system", "my-topic1", new Partition(i)));
-      }
-    } };
-
-  @Test public void testUserTask() throws Exception {
-    Config mockConfig = mock(Config.class);
-    TaskContext mockContext = mock(TaskContext.class);
-    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
-    StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(this.userTask);
-    Field pipelineMapFld = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
-    pipelineMapFld.setAccessible(true);
-    Map<SystemStreamPartition, ChainedOperators> pipelineMap =
-        (Map<SystemStreamPartition, ChainedOperators>) pipelineMapFld.get(adaptorTask);
-
-    adaptorTask.init(mockConfig, mockContext);
-    assertEquals(pipelineMap.size(), 4);
-    this.inputPartitions.forEach(partition -> {
-        assertNotNull(pipelineMap.get(partition));
-      });
-  }
-
-  @Test public void testSplitTask() throws Exception {
-    Config mockConfig = mock(Config.class);
-    TaskContext mockContext = mock(TaskContext.class);
-    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
-    StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(this.splitTask);
-    Field pipelineMapFld = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
-    pipelineMapFld.setAccessible(true);
-    Map<SystemStreamPartition, ChainedOperators> pipelineMap =
-        (Map<SystemStreamPartition, ChainedOperators>) pipelineMapFld.get(adaptorTask);
-
-    adaptorTask.init(mockConfig, mockContext);
-    assertEquals(pipelineMap.size(), 4);
-    this.inputPartitions.forEach(partition -> {
-        assertNotNull(pipelineMap.get(partition));
-      });
-  }
-
-  @Test public void testJoinTask() throws Exception {
-    Config mockConfig = mock(Config.class);
-    TaskContext mockContext = mock(TaskContext.class);
-    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
-    StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(this.joinTask);
-    Field pipelineMapFld = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
-    pipelineMapFld.setAccessible(true);
-    Map<SystemStreamPartition, ChainedOperators> pipelineMap =
-        (Map<SystemStreamPartition, ChainedOperators>) pipelineMapFld.get(adaptorTask);
-
-    adaptorTask.init(mockConfig, mockContext);
-    assertEquals(pipelineMap.size(), 4);
-    this.inputPartitions.forEach(partition -> {
-        assertNotNull(pipelineMap.get(partition));
-      });
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java b/samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java
deleted file mode 100644
index de7bba5..0000000
--- a/samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java
+++ /dev/null
@@ -1,71 +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.samza.task;
-
-import org.apache.samza.operators.MessageStreams.SystemMessageStream;
-import org.apache.samza.operators.TriggerBuilder;
-import org.apache.samza.operators.Windows;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.data.Offset;
-import org.apache.samza.operators.task.StreamOperatorTask;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.util.Collection;
-
-
-/**
- * Example implementation of a simple user-defined tasks w/ window operators
- *
- */
-public class WindowOperatorTask implements StreamOperatorTask {
-  class MessageType {
-    String field1;
-    String field2;
-  }
-
-  class JsonMessage extends InputJsonSystemMessage<MessageType> {
-
-    JsonMessage(String key, MessageType data, Offset offset, long timestamp, SystemStreamPartition partition) {
-      super(key, data, offset, timestamp, partition);
-    }
-  }
-
-  @Override public void initOperators(Collection<SystemMessageStream> sources) {
-    sources.forEach(source ->
-      source.map(m1 ->
-        new JsonMessage(
-          this.myMessageKeyFunction(m1),
-          (MessageType) m1.getMessage(),
-          m1.getOffset(),
-          m1.getTimestamp(),
-          m1.getSystemStreamPartition())).
-        window(
-          Windows.<JsonMessage, String>intoSessionCounter(
-              m -> String.format("%s-%s", m.getMessage().field1, m.getMessage().field2)).
-            setTriggers(TriggerBuilder.<JsonMessage, Integer>earlyTriggerWhenExceedWndLen(100).
-              addTimeoutSinceLastMessage(30000)))
-    );
-  }
-
-  String myMessageKeyFunction(Message<Object, Object> m) {
-    return m.getKey().toString();
-  }
-
-}


[2/6] samza git commit: SAMZA-1054: Refactor Operator APIs

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
new file mode 100644
index 0000000..2ad6461
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
@@ -0,0 +1,203 @@
+/*
+ * 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.samza.operators;
+
+import org.apache.samza.operators.functions.FilterFunction;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.functions.JoinFunction;
+import org.apache.samza.operators.functions.MapFunction;
+import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.operators.spec.OperatorSpec;
+import org.apache.samza.operators.spec.PartialJoinOperatorSpec;
+import org.apache.samza.operators.spec.SinkOperatorSpec;
+import org.apache.samza.operators.spec.StreamOperatorSpec;
+import org.apache.samza.operators.spec.WindowOperatorSpec;
+import org.apache.samza.operators.windows.SessionWindow;
+import org.apache.samza.operators.windows.WindowFn;
+import org.apache.samza.operators.windows.WindowOutput;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.task.TaskCoordinator;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
+public class TestMessageStreamImpl {
+
+  @Test
+  public void testMap() {
+    MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>();
+    MapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> xMap =
+        m -> new TestOutputMessageEnvelope(m.getKey(), m.getMessage().getValue().length() + 1);
+    MessageStream<TestOutputMessageEnvelope> outputStream = inputStream.map(xMap);
+    Collection<OperatorSpec> subs = inputStream.getRegisteredOperatorSpecs();
+    assertEquals(subs.size(), 1);
+    OperatorSpec<TestOutputMessageEnvelope> mapOp = subs.iterator().next();
+    assertTrue(mapOp instanceof StreamOperatorSpec);
+    assertEquals(mapOp.getOutputStream(), outputStream);
+    // assert that the transformation function is what we defined above
+    TestMessageEnvelope xTestMsg = mock(TestMessageEnvelope.class);
+    TestMessageEnvelope.MessageType mockInnerTestMessage = mock(TestMessageEnvelope.MessageType.class);
+    when(xTestMsg.getKey()).thenReturn("test-msg-key");
+    when(xTestMsg.getMessage()).thenReturn(mockInnerTestMessage);
+    when(mockInnerTestMessage.getValue()).thenReturn("123456789");
+
+    Collection<TestOutputMessageEnvelope> cOutputMsg = ((StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope>) mapOp).getTransformFn().apply(xTestMsg);
+    assertEquals(cOutputMsg.size(), 1);
+    TestOutputMessageEnvelope outputMessage = cOutputMsg.iterator().next();
+    assertEquals(outputMessage.getKey(), xTestMsg.getKey());
+    assertEquals(outputMessage.getMessage(), Integer.valueOf(xTestMsg.getMessage().getValue().length() + 1));
+  }
+
+  @Test
+  public void testFlatMap() {
+    MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>();
+    Set<TestOutputMessageEnvelope> flatOuts = new HashSet<TestOutputMessageEnvelope>() { {
+        this.add(mock(TestOutputMessageEnvelope.class));
+        this.add(mock(TestOutputMessageEnvelope.class));
+        this.add(mock(TestOutputMessageEnvelope.class));
+      } };
+    FlatMapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> xFlatMap = m -> flatOuts;
+    MessageStream<TestOutputMessageEnvelope> outputStream = inputStream.flatMap(xFlatMap);
+    Collection<OperatorSpec> subs = inputStream.getRegisteredOperatorSpecs();
+    assertEquals(subs.size(), 1);
+    OperatorSpec<TestOutputMessageEnvelope> flatMapOp = subs.iterator().next();
+    assertTrue(flatMapOp instanceof StreamOperatorSpec);
+    assertEquals(flatMapOp.getOutputStream(), outputStream);
+    // assert that the transformation function is what we defined above
+    assertEquals(((StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope>) flatMapOp).getTransformFn(), xFlatMap);
+  }
+
+  @Test
+  public void testFilter() {
+    MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>();
+    FilterFunction<TestMessageEnvelope> xFilter = m -> m.getMessage().getEventTime() > 123456L;
+    MessageStream<TestMessageEnvelope> outputStream = inputStream.filter(xFilter);
+    Collection<OperatorSpec> subs = inputStream.getRegisteredOperatorSpecs();
+    assertEquals(subs.size(), 1);
+    OperatorSpec<TestMessageEnvelope> filterOp = subs.iterator().next();
+    assertTrue(filterOp instanceof StreamOperatorSpec);
+    assertEquals(filterOp.getOutputStream(), outputStream);
+    // assert that the transformation function is what we defined above
+    FlatMapFunction<TestMessageEnvelope, TestMessageEnvelope> txfmFn = ((StreamOperatorSpec<TestMessageEnvelope, TestMessageEnvelope>) filterOp).getTransformFn();
+    TestMessageEnvelope mockMsg = mock(TestMessageEnvelope.class);
+    TestMessageEnvelope.MessageType mockInnerTestMessage = mock(TestMessageEnvelope.MessageType.class);
+    when(mockMsg.getMessage()).thenReturn(mockInnerTestMessage);
+    when(mockInnerTestMessage.getEventTime()).thenReturn(11111L);
+    Collection<TestMessageEnvelope> output = txfmFn.apply(mockMsg);
+    assertTrue(output.isEmpty());
+    when(mockMsg.getMessage()).thenReturn(mockInnerTestMessage);
+    when(mockInnerTestMessage.getEventTime()).thenReturn(999999L);
+    output = txfmFn.apply(mockMsg);
+    assertEquals(output.size(), 1);
+    assertEquals(output.iterator().next(), mockMsg);
+  }
+
+  @Test
+  public void testSink() {
+    MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>();
+    SinkFunction<TestMessageEnvelope> xSink = (m, mc, tc) -> {
+      mc.send(new OutgoingMessageEnvelope(new SystemStream("test-sys", "test-stream"), m.getMessage()));
+      tc.commit(TaskCoordinator.RequestScope.CURRENT_TASK);
+    };
+    inputStream.sink(xSink);
+    Collection<OperatorSpec> subs = inputStream.getRegisteredOperatorSpecs();
+    assertEquals(subs.size(), 1);
+    OperatorSpec<TestMessageEnvelope> sinkOp = subs.iterator().next();
+    assertTrue(sinkOp instanceof SinkOperatorSpec);
+    assertEquals(((SinkOperatorSpec) sinkOp).getSinkFn(), xSink);
+    assertNull(((SinkOperatorSpec) sinkOp).getOutputStream());
+  }
+
+  @Test
+  public void testWindow() {
+    MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>();
+    SessionWindow<TestMessageEnvelope, String, Integer> window = mock(SessionWindow.class);
+    doReturn(mock(WindowFn.class)).when(window).getInternalWindowFn();
+    MessageStream<WindowOutput<String, Integer>> outStream = inputStream.window(window);
+    Collection<OperatorSpec> subs = inputStream.getRegisteredOperatorSpecs();
+    assertEquals(subs.size(), 1);
+    OperatorSpec<TestMessageEnvelope> wndOp = subs.iterator().next();
+    assertTrue(wndOp instanceof WindowOperatorSpec);
+    assertEquals(((WindowOperatorSpec) wndOp).getOutputStream(), outStream);
+  }
+
+  @Test
+  public void testJoin() {
+    MessageStreamImpl<TestMessageEnvelope> source1 = new MessageStreamImpl<>();
+    MessageStreamImpl<TestMessageEnvelope> source2 = new MessageStreamImpl<>();
+    JoinFunction<TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope> joiner =
+        (m1, m2) -> new TestOutputMessageEnvelope(m1.getKey(), m1.getMessage().getValue().length() + m2.getMessage().getValue().length());
+    MessageStream<TestOutputMessageEnvelope> joinOutput = source1.join(source2, joiner);
+    Collection<OperatorSpec> subs = source1.getRegisteredOperatorSpecs();
+    assertEquals(subs.size(), 1);
+    OperatorSpec<TestMessageEnvelope> joinOp1 = subs.iterator().next();
+    assertTrue(joinOp1 instanceof PartialJoinOperatorSpec);
+    assertEquals(((PartialJoinOperatorSpec) joinOp1).getOutputStream(), joinOutput);
+    subs = source2.getRegisteredOperatorSpecs();
+    assertEquals(subs.size(), 1);
+    OperatorSpec<TestMessageEnvelope> joinOp2 = subs.iterator().next();
+    assertTrue(joinOp2 instanceof PartialJoinOperatorSpec);
+    assertEquals(((PartialJoinOperatorSpec) joinOp2).getOutputStream(), joinOutput);
+    TestMessageEnvelope joinMsg1 = new TestMessageEnvelope("test-join-1", "join-msg-001", 11111L);
+    TestMessageEnvelope joinMsg2 = new TestMessageEnvelope("test-join-2", "join-msg-002", 22222L);
+    TestOutputMessageEnvelope xOut = (TestOutputMessageEnvelope) ((PartialJoinOperatorSpec) joinOp1).getTransformFn().apply(joinMsg1, joinMsg2);
+    assertEquals(xOut.getKey(), "test-join-1");
+    assertEquals(xOut.getMessage(), Integer.valueOf(24));
+    xOut = (TestOutputMessageEnvelope) ((PartialJoinOperatorSpec) joinOp2).getTransformFn().apply(joinMsg2, joinMsg1);
+    assertEquals(xOut.getKey(), "test-join-1");
+    assertEquals(xOut.getMessage(), Integer.valueOf(24));
+  }
+
+  @Test
+  public void testMerge() {
+    MessageStream<TestMessageEnvelope> merge1 = new MessageStreamImpl<>();
+    Collection<MessageStream<TestMessageEnvelope>> others = new ArrayList<MessageStream<TestMessageEnvelope>>() { {
+        this.add(new MessageStreamImpl<>());
+        this.add(new MessageStreamImpl<>());
+      } };
+    MessageStream<TestMessageEnvelope> mergeOutput = merge1.merge(others);
+    validateMergeOperator(merge1, mergeOutput);
+
+    others.forEach(merge -> validateMergeOperator(merge, mergeOutput));
+  }
+
+  private void validateMergeOperator(MessageStream<TestMessageEnvelope> mergeSource, MessageStream<TestMessageEnvelope> mergeOutput) {
+    Collection<OperatorSpec> subs = ((MessageStreamImpl<TestMessageEnvelope>) mergeSource).getRegisteredOperatorSpecs();
+    assertEquals(subs.size(), 1);
+    OperatorSpec<TestMessageEnvelope> mergeOp = subs.iterator().next();
+    assertTrue(mergeOp instanceof StreamOperatorSpec);
+    assertEquals(((StreamOperatorSpec) mergeOp).getOutputStream(), mergeOutput);
+    TestMessageEnvelope mockMsg = mock(TestMessageEnvelope.class);
+    Collection<TestMessageEnvelope> outputs = ((StreamOperatorSpec<TestMessageEnvelope, TestMessageEnvelope>) mergeOp).getTransformFn().apply(mockMsg);
+    assertEquals(outputs.size(), 1);
+    assertEquals(outputs.iterator().next(), mockMsg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/TestStateStoreImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/TestStateStoreImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/TestStateStoreImpl.java
new file mode 100644
index 0000000..8fa7ccc
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/TestStateStoreImpl.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.samza.operators;
+
+import org.apache.samza.operators.windows.StoreFunctions;
+import org.apache.samza.operators.windows.WindowState;
+import org.apache.samza.storage.kv.Entry;
+import org.apache.samza.storage.kv.KeyValueStore;
+import org.apache.samza.task.TaskContext;
+import org.junit.Test;
+
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+
+public class TestStateStoreImpl {
+  @Test
+  public void testStateStoreImpl() {
+    StoreFunctions<TestMessageEnvelope, String, WindowState> mockStoreFunctions = mock(StoreFunctions.class);
+    // test constructor
+    StateStoreImpl<TestMessageEnvelope, String, WindowState> storeImpl = new StateStoreImpl<>(mockStoreFunctions, "myStoreName");
+    TaskContext mockContext = mock(TaskContext.class);
+    KeyValueStore<String, WindowState> mockKvStore = mock(KeyValueStore.class);
+    when(mockContext.getStore("myStoreName")).thenReturn(mockKvStore);
+    // test init()
+    storeImpl.init(mockContext);
+    verify(mockContext, times(1)).getStore("myStoreName");
+    Function<TestMessageEnvelope, String> wndKeyFn = mock(Function.class);
+    when(mockStoreFunctions.getStoreKeyFn()).thenReturn(wndKeyFn);
+    TestMessageEnvelope mockMsg = mock(TestMessageEnvelope.class);
+    when(wndKeyFn.apply(mockMsg)).thenReturn("myKey");
+    WindowState mockState = mock(WindowState.class);
+    when(mockKvStore.get("myKey")).thenReturn(mockState);
+    // test getState()
+    Entry<String, WindowState> storeEntry = storeImpl.getState(mockMsg);
+    assertEquals(storeEntry.getKey(), "myKey");
+    assertEquals(storeEntry.getValue(), mockState);
+    verify(wndKeyFn, times(1)).apply(mockMsg);
+    verify(mockKvStore, times(1)).get("myKey");
+    Entry<String, WindowState> oldEntry = new Entry<>("myKey", mockState);
+    WindowState mockNewState = mock(WindowState.class);
+    BiFunction<TestMessageEnvelope, WindowState, WindowState> mockUpdaterFn = mock(BiFunction.class);
+    when(mockStoreFunctions.getStateUpdaterFn()).thenReturn(mockUpdaterFn);
+    when(mockUpdaterFn.apply(mockMsg, mockState)).thenReturn(mockNewState);
+    // test updateState()
+    Entry<String, WindowState> newEntry = storeImpl.updateState(mockMsg, oldEntry);
+    assertEquals(newEntry.getKey(), "myKey");
+    assertEquals(newEntry.getValue(), mockNewState);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/WindowTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/WindowTask.java b/samza-operator/src/test/java/org/apache/samza/operators/WindowTask.java
new file mode 100644
index 0000000..f33510e
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/WindowTask.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.samza.operators;
+
+import org.apache.samza.operators.data.IncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.data.Offset;
+import org.apache.samza.operators.windows.TriggerBuilder;
+import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.util.Map;
+
+
+/**
+ * Example implementation of a simple user-defined tasks w/ window operators
+ *
+ */
+public class WindowTask implements StreamOperatorTask {
+  class MessageType {
+    String field1;
+    String field2;
+  }
+
+  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
+
+    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
+      super(key, data, offset, partition);
+    }
+  }
+
+  @Override public void transform(Map<SystemStreamPartition, MessageStream<IncomingSystemMessageEnvelope>> messageStreams) {
+    messageStreams.values().forEach(source ->
+      source.map(m1 ->
+        new JsonMessageEnvelope(
+          this.myMessageKeyFunction(m1),
+          (MessageType) m1.getMessage(),
+          m1.getOffset(),
+          m1.getSystemStreamPartition())).
+        window(
+          Windows.<JsonMessageEnvelope, String>intoSessionCounter(
+              m -> String.format("%s-%s", m.getMessage().field1, m.getMessage().field2)).
+            setTriggers(TriggerBuilder.<JsonMessageEnvelope, Integer>earlyTriggerWhenExceedWndLen(100).
+              addTimeoutSinceLastMessage(30000)))
+    );
+  }
+
+  String myMessageKeyFunction(MessageEnvelope<Object, Object> m) {
+    return m.getKey().toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/data/JsonIncomingSystemMessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/data/JsonIncomingSystemMessageEnvelope.java b/samza-operator/src/test/java/org/apache/samza/operators/data/JsonIncomingSystemMessageEnvelope.java
new file mode 100644
index 0000000..9a425d1
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/data/JsonIncomingSystemMessageEnvelope.java
@@ -0,0 +1,60 @@
+/*
+ * 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.samza.operators.data;
+
+import org.apache.samza.system.SystemStreamPartition;
+
+
+/**
+ * Example input {@link MessageEnvelope} w/ Json message and string as the key.
+ */
+
+public class JsonIncomingSystemMessageEnvelope<T> implements MessageEnvelope<String, T> {
+
+  private final String key;
+  private final T data;
+  private final Offset offset;
+  private final SystemStreamPartition partition;
+
+  public JsonIncomingSystemMessageEnvelope(String key, T data, Offset offset, SystemStreamPartition partition) {
+    this.key = key;
+    this.data = data;
+    this.offset = offset;
+    this.partition = partition;
+  }
+
+  @Override
+  public T getMessage() {
+    return this.data;
+  }
+
+  @Override
+  public String getKey() {
+    return this.key;
+  }
+
+  public Offset getOffset() {
+    return this.offset;
+  }
+
+  public SystemStreamPartition getSystemStreamPartition() {
+    return this.partition;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/TestChainedOperators.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestChainedOperators.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestChainedOperators.java
deleted file mode 100644
index e3a70e8..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestChainedOperators.java
+++ /dev/null
@@ -1,129 +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.samza.operators.impl;
-
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.TestMessage;
-import org.apache.samza.operators.TestOutputMessage;
-import org.apache.samza.operators.Windows;
-import org.apache.samza.task.TaskContext;
-import org.junit.Before;
-import org.junit.Test;
-import org.reactivestreams.Subscriber;
-
-import java.lang.reflect.Field;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.Set;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.mock;
-
-
-public class TestChainedOperators {
-  Field subsField = null;
-  Field opSubsField = null;
-
-  @Before public void prep() throws NoSuchFieldException {
-    subsField = ChainedOperators.class.getDeclaredField("subscribers");
-    subsField.setAccessible(true);
-    opSubsField = OperatorImpl.class.getDeclaredField("subscribers");
-    opSubsField.setAccessible(true);
-  }
-
-  @Test public void testCreate() {
-    // test creation of empty chain
-    MessageStream<TestMessage> testStream = new MessageStream<>();
-    TaskContext mockContext = mock(TaskContext.class);
-    ChainedOperators<TestMessage> operatorChain = ChainedOperators.create(testStream, mockContext);
-    assertTrue(operatorChain != null);
-  }
-
-  @Test public void testLinearChain() throws IllegalAccessException {
-    // test creation of linear chain
-    MessageStream<TestMessage> testInput = new MessageStream<>();
-    TaskContext mockContext = mock(TaskContext.class);
-    testInput.map(m -> m).window(Windows.intoSessionCounter(TestMessage::getKey));
-    ChainedOperators<TestMessage> operatorChain = ChainedOperators.create(testInput, mockContext);
-    Set<OperatorImpl> subsSet = (Set<OperatorImpl>) subsField.get(operatorChain);
-    assertEquals(subsSet.size(), 1);
-    OperatorImpl<TestMessage, TestMessage> firstOpImpl = subsSet.iterator().next();
-    Set<Subscriber<? super ProcessorContext<TestMessage>>> subsOps = (Set<Subscriber<? super ProcessorContext<TestMessage>>>) opSubsField.get(firstOpImpl);
-    assertEquals(subsOps.size(), 1);
-    Subscriber<? super ProcessorContext<TestMessage>> wndOpImpl = subsOps.iterator().next();
-    subsOps = (Set<Subscriber<? super ProcessorContext<TestMessage>>>) opSubsField.get(wndOpImpl);
-    assertEquals(subsOps.size(), 0);
-  }
-
-  @Test public void testBroadcastChain() throws IllegalAccessException {
-    // test creation of broadcast chain
-    MessageStream<TestMessage> testInput = new MessageStream<>();
-    TaskContext mockContext = mock(TaskContext.class);
-    testInput.filter(m -> m.getTimestamp() > 123456L).flatMap(m -> new ArrayList() { { this.add(m); this.add(m); } });
-    testInput.filter(m -> m.getTimestamp() < 123456L).map(m -> m);
-    ChainedOperators<TestMessage> operatorChain = ChainedOperators.create(testInput, mockContext);
-    Set<OperatorImpl> subsSet = (Set<OperatorImpl>) subsField.get(operatorChain);
-    assertEquals(subsSet.size(), 2);
-    Iterator<OperatorImpl> iter = subsSet.iterator();
-    // check the first branch w/ flatMap
-    OperatorImpl<TestMessage, TestMessage> opImpl = iter.next();
-    Set<Subscriber<? super ProcessorContext<TestMessage>>> subsOps = (Set<Subscriber<? super ProcessorContext<TestMessage>>>) opSubsField.get(opImpl);
-    assertEquals(subsOps.size(), 1);
-    Subscriber<? super ProcessorContext<TestMessage>> flatMapImpl = subsOps.iterator().next();
-    subsOps = (Set<Subscriber<? super ProcessorContext<TestMessage>>>) opSubsField.get(flatMapImpl);
-    assertEquals(subsOps.size(), 0);
-    // check the second branch w/ map
-    opImpl = iter.next();
-    subsOps = (Set<Subscriber<? super ProcessorContext<TestMessage>>>) opSubsField.get(opImpl);
-    assertEquals(subsOps.size(), 1);
-    Subscriber<? super ProcessorContext<TestMessage>> mapImpl = subsOps.iterator().next();
-    subsOps = (Set<Subscriber<? super ProcessorContext<TestMessage>>>) opSubsField.get(mapImpl);
-    assertEquals(subsOps.size(), 0);
-  }
-
-  @Test public void testJoinChain() throws IllegalAccessException {
-    // test creation of join chain
-    MessageStream<TestMessage> input1 = new MessageStream<>();
-    MessageStream<TestMessage> input2 = new MessageStream<>();
-    TaskContext mockContext = mock(TaskContext.class);
-    input1.join(input2, (m1, m2) -> new TestOutputMessage(m1.getKey(), m1.getMessage().length() + m2.getMessage().length(), m1.getTimestamp())).map(m -> m);
-    // now, we create chained operators from each input sources
-    ChainedOperators<TestMessage> chain1 = ChainedOperators.create(input1, mockContext);
-    ChainedOperators<TestMessage> chain2 = ChainedOperators.create(input2, mockContext);
-    // check that those two chains will merge at map operator
-    // first branch of the join
-    Set<OperatorImpl> subsSet = (Set<OperatorImpl>) subsField.get(chain1);
-    assertEquals(subsSet.size(), 1);
-    OperatorImpl<TestMessage, TestOutputMessage> joinOp1 = subsSet.iterator().next();
-    Set<Subscriber<? super ProcessorContext<TestOutputMessage>>> subsOps = (Set<Subscriber<? super ProcessorContext<TestOutputMessage>>>) opSubsField.get(joinOp1);
-    assertEquals(subsOps.size(), 1);
-    // the map operator consumes the common join output, where two branches merge
-    Subscriber<? super ProcessorContext<TestOutputMessage>> mapImpl = subsOps.iterator().next();
-    // second branch of the join
-    subsSet = (Set<OperatorImpl>) subsField.get(chain2);
-    assertEquals(subsSet.size(), 1);
-    OperatorImpl<TestMessage, TestOutputMessage> joinOp2 = subsSet.iterator().next();
-    assertNotSame(joinOp1, joinOp2);
-    subsOps = (Set<Subscriber<? super ProcessorContext<TestOutputMessage>>>) opSubsField.get(joinOp2);
-    assertEquals(subsOps.size(), 1);
-    // make sure that the map operator is the same
-    assertEquals(mapImpl, subsOps.iterator().next());
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorFactory.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorFactory.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorFactory.java
deleted file mode 100644
index cb4576c..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorFactory.java
+++ /dev/null
@@ -1,93 +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.samza.operators.impl;
-
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.TestMessage;
-import org.apache.samza.operators.TestOutputMessage;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.impl.join.PartialJoinOpImpl;
-import org.apache.samza.operators.impl.window.SessionWindowImpl;
-import org.apache.samza.operators.internal.Operators.PartialJoinOperator;
-import org.apache.samza.operators.internal.Operators.SinkOperator;
-import org.apache.samza.operators.internal.Operators.StreamOperator;
-import org.apache.samza.operators.internal.Operators.WindowOperator;
-import org.apache.samza.storage.kv.Entry;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-import org.junit.Test;
-
-import java.lang.reflect.Field;
-import java.util.Collection;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-
-public class TestOperatorFactory {
-
-  @Test public void testGetOperator() throws NoSuchFieldException, IllegalAccessException {
-    // get window operator
-    WindowOperator mockWnd = mock(WindowOperator.class);
-    Entry<OperatorImpl<TestMessage, ? extends Message>, Boolean>
-        factoryEntry = OperatorFactory.<TestMessage, TestOutputMessage>getOperator(mockWnd);
-    assertFalse(factoryEntry.getValue());
-    OperatorImpl<TestMessage, TestOutputMessage> opImpl = (OperatorImpl<TestMessage, TestOutputMessage>) factoryEntry.getKey();
-    assertTrue(opImpl instanceof SessionWindowImpl);
-    Field sessWndField = SessionWindowImpl.class.getDeclaredField("sessWnd");
-    sessWndField.setAccessible(true);
-    WindowOperator sessWnd = (WindowOperator) sessWndField.get(opImpl);
-    assertEquals(sessWnd, mockWnd);
-
-    // get simple operator
-    StreamOperator<TestMessage, TestOutputMessage> mockSimpleOp = mock(StreamOperator.class);
-    Function<TestMessage, Collection<TestOutputMessage>>  mockTxfmFn = mock(Function.class);
-    when(mockSimpleOp.getFunction()).thenReturn(mockTxfmFn);
-    factoryEntry = OperatorFactory.<TestMessage, TestOutputMessage>getOperator(mockSimpleOp);
-    opImpl = (OperatorImpl<TestMessage, TestOutputMessage>) factoryEntry.getKey();
-    assertTrue(opImpl instanceof SimpleOperatorImpl);
-    Field txfmFnField = SimpleOperatorImpl.class.getDeclaredField("transformFn");
-    txfmFnField.setAccessible(true);
-    assertEquals(mockTxfmFn, txfmFnField.get(opImpl));
-
-    // get sink operator
-    MessageStream.VoidFunction3<TestMessage, MessageCollector, TaskCoordinator> sinkFn = (m, mc, tc) -> { };
-    SinkOperator<TestMessage> sinkOp = mock(SinkOperator.class);
-    when(sinkOp.getFunction()).thenReturn(sinkFn);
-    factoryEntry = OperatorFactory.<TestMessage, TestOutputMessage>getOperator(sinkOp);
-    opImpl = (OperatorImpl<TestMessage, TestOutputMessage>) factoryEntry.getKey();
-    assertTrue(opImpl instanceof SinkOperatorImpl);
-    Field sinkFnField = SinkOperatorImpl.class.getDeclaredField("sinkFunc");
-    sinkFnField.setAccessible(true);
-    assertEquals(sinkFn, sinkFnField.get(opImpl));
-
-    // get join operator
-    PartialJoinOperator<TestMessage, String, TestMessage, TestOutputMessage> joinOp = mock(PartialJoinOperator.class);
-    TestOutputMessage mockOutput = mock(TestOutputMessage.class);
-    BiFunction<TestMessage, TestMessage, TestOutputMessage> joinFn = (m1, m2) -> mockOutput;
-    when(joinOp.getFunction()).thenReturn(joinFn);
-    factoryEntry = OperatorFactory.<TestMessage, TestOutputMessage>getOperator(joinOp);
-    opImpl = (OperatorImpl<TestMessage, TestOutputMessage>) factoryEntry.getKey();
-    assertTrue(opImpl instanceof PartialJoinOpImpl);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
index 4bd467d..361972e 100644
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
@@ -18,52 +18,54 @@
  */
 package org.apache.samza.operators.impl;
 
-import org.apache.samza.operators.TestMessage;
-import org.apache.samza.operators.TestOutputMessage;
+import org.apache.samza.operators.TestMessageEnvelope;
+import org.apache.samza.operators.TestOutputMessageEnvelope;
 import org.apache.samza.task.MessageCollector;
 import org.apache.samza.task.TaskCoordinator;
+import org.hamcrest.core.IsEqual;
 import org.junit.Test;
-import org.mockito.ArgumentMatcher;
-import org.reactivestreams.Subscriber;
 
 import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.argThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
 
 
 public class TestOperatorImpl {
 
-  TestMessage curInputMsg;
+  TestMessageEnvelope curInputMsg;
   MessageCollector curCollector;
   TaskCoordinator curCoordinator;
 
-  @Test public void testSubscribers() {
+  @Test
+  public void testSubscribers() {
     this.curInputMsg = null;
     this.curCollector = null;
     this.curCoordinator = null;
-    OperatorImpl<TestMessage, TestOutputMessage> opImpl = new OperatorImpl<TestMessage, TestOutputMessage>() {
-      @Override protected void onNext(TestMessage message, MessageCollector collector, TaskCoordinator coordinator) {
+    OperatorImpl<TestMessageEnvelope, TestOutputMessageEnvelope> opImpl = new OperatorImpl<TestMessageEnvelope, TestOutputMessageEnvelope>() {
+      @Override
+      public void onNext(TestMessageEnvelope message, MessageCollector collector, TaskCoordinator coordinator) {
         TestOperatorImpl.this.curInputMsg = message;
         TestOperatorImpl.this.curCollector = collector;
         TestOperatorImpl.this.curCoordinator = coordinator;
       }
     };
-    // verify subscribe() added the mockSub and nextProcessors() invoked the mockSub.onNext()
-    Subscriber<ProcessorContext<TestOutputMessage>> mockSub = mock(Subscriber.class);
-    opImpl.subscribe(mockSub);
-    TestOutputMessage xOutput = mock(TestOutputMessage.class);
+    // verify registerNextOperator() added the mockSub and propagateResult() invoked the mockSub.onNext()
+    OperatorImpl mockSub = mock(OperatorImpl.class);
+    opImpl.registerNextOperator(mockSub);
+    TestOutputMessageEnvelope xOutput = mock(TestOutputMessageEnvelope.class);
     MessageCollector mockCollector = mock(MessageCollector.class);
     TaskCoordinator mockCoordinator = mock(TaskCoordinator.class);
-    opImpl.nextProcessors(xOutput, mockCollector, mockCoordinator);
-    verify(mockSub, times(1)).onNext(argThat(new ArgumentMatcher<ProcessorContext<TestOutputMessage>>() {
-      @Override public boolean matches(Object argument) {
-        ProcessorContext<TestOutputMessage> pCntx = (ProcessorContext<TestOutputMessage>) argument;
-        return pCntx.getMessage().equals(xOutput) && pCntx.getCoordinator().equals(mockCoordinator) && pCntx.getCollector().equals(mockCollector);
-      }
-    }));
+    opImpl.propagateResult(xOutput, mockCollector, mockCoordinator);
+    verify(mockSub, times(1)).onNext(
+        argThat(new IsEqual<>(xOutput)),
+        argThat(new IsEqual<>(mockCollector)),
+        argThat(new IsEqual<>(mockCoordinator))
+    );
     // verify onNext() is invoked correctly
-    TestMessage mockInput = mock(TestMessage.class);
-    ProcessorContext<TestMessage> inCntx = new ProcessorContext<>(mockInput, mockCollector, mockCoordinator);
-    opImpl.onNext(inCntx);
+    TestMessageEnvelope mockInput = mock(TestMessageEnvelope.class);
+    opImpl.onNext(mockInput, mockCollector, mockCoordinator);
     assertEquals(mockInput, this.curInputMsg);
     assertEquals(mockCollector, this.curCollector);
     assertEquals(mockCoordinator, this.curCoordinator);

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java
new file mode 100644
index 0000000..1d0d547
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java
@@ -0,0 +1,183 @@
+/*
+ * 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.samza.operators.impl;
+
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.TestMessageEnvelope;
+import org.apache.samza.operators.TestOutputMessageEnvelope;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.operators.spec.PartialJoinOperatorSpec;
+import org.apache.samza.operators.spec.SinkOperatorSpec;
+import org.apache.samza.operators.spec.StreamOperatorSpec;
+import org.apache.samza.operators.spec.WindowOperatorSpec;
+import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.task.TaskContext;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
+public class TestOperatorImpls {
+  Field nextOperatorsField = null;
+
+  @Before
+  public void prep() throws NoSuchFieldException {
+    nextOperatorsField = OperatorImpl.class.getDeclaredField("nextOperators");
+    nextOperatorsField.setAccessible(true);
+  }
+  
+  @Test
+  public void testCreateOperator() throws NoSuchFieldException, IllegalAccessException {
+    // get window operator
+    WindowOperatorSpec mockWnd = mock(WindowOperatorSpec.class);
+    OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope> opImpl = OperatorImpls.createOperatorImpl(mockWnd);
+    assertTrue(opImpl instanceof SessionWindowOperatorImpl);
+    Field sessWndField = SessionWindowOperatorImpl.class.getDeclaredField("windowSpec");
+    sessWndField.setAccessible(true);
+    WindowOperatorSpec sessWnd = (WindowOperatorSpec) sessWndField.get(opImpl);
+    assertEquals(sessWnd, mockWnd);
+
+    // get simple operator
+    StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> mockSimpleOp = mock(StreamOperatorSpec.class);
+    FlatMapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> mockTxfmFn = mock(FlatMapFunction.class);
+    when(mockSimpleOp.getTransformFn()).thenReturn(mockTxfmFn);
+    opImpl = OperatorImpls.createOperatorImpl(mockSimpleOp);
+    assertTrue(opImpl instanceof StreamOperatorImpl);
+    Field txfmFnField = StreamOperatorImpl.class.getDeclaredField("transformFn");
+    txfmFnField.setAccessible(true);
+    assertEquals(mockTxfmFn, txfmFnField.get(opImpl));
+
+    // get sink operator
+    SinkFunction<TestMessageEnvelope> sinkFn = (m, mc, tc) -> { };
+    SinkOperatorSpec<TestMessageEnvelope> sinkOp = mock(SinkOperatorSpec.class);
+    when(sinkOp.getSinkFn()).thenReturn(sinkFn);
+    opImpl = OperatorImpls.createOperatorImpl(sinkOp);
+    assertTrue(opImpl instanceof SinkOperatorImpl);
+    Field sinkFnField = SinkOperatorImpl.class.getDeclaredField("sinkFn");
+    sinkFnField.setAccessible(true);
+    assertEquals(sinkFn, sinkFnField.get(opImpl));
+
+    // get join operator
+    PartialJoinOperatorSpec<TestMessageEnvelope, String, TestMessageEnvelope, TestOutputMessageEnvelope> joinOp = mock(PartialJoinOperatorSpec.class);
+    TestOutputMessageEnvelope mockOutput = mock(TestOutputMessageEnvelope.class);
+    BiFunction<TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope> joinFn = (m1, m2) -> mockOutput;
+    when(joinOp.getTransformFn()).thenReturn(joinFn);
+    opImpl = OperatorImpls.createOperatorImpl(joinOp);
+    assertTrue(opImpl instanceof PartialJoinOperatorImpl);
+  }
+
+  @Test
+  public void testEmptyChain() {
+    // test creation of empty chain
+    MessageStreamImpl<TestMessageEnvelope> testStream = new MessageStreamImpl<>();
+    TaskContext mockContext = mock(TaskContext.class);
+    RootOperatorImpl operatorChain = OperatorImpls.createOperatorImpls(testStream, mockContext);
+    assertTrue(operatorChain != null);
+  }
+
+  @Test
+  public void testLinearChain() throws IllegalAccessException {
+    // test creation of linear chain
+    MessageStreamImpl<TestMessageEnvelope> testInput = new MessageStreamImpl<>();
+    TaskContext mockContext = mock(TaskContext.class);
+    testInput.map(m -> m).window(Windows.intoSessionCounter(TestMessageEnvelope::getKey));
+    RootOperatorImpl operatorChain = OperatorImpls.createOperatorImpls(testInput, mockContext);
+    Set<OperatorImpl> subsSet = (Set<OperatorImpl>) nextOperatorsField.get(operatorChain);
+    assertEquals(subsSet.size(), 1);
+    OperatorImpl<TestMessageEnvelope, TestMessageEnvelope> firstOpImpl = subsSet.iterator().next();
+    Set<OperatorImpl> subsOps = (Set<OperatorImpl>) nextOperatorsField.get(firstOpImpl);
+    assertEquals(subsOps.size(), 1);
+    OperatorImpl wndOpImpl = subsOps.iterator().next();
+    subsOps = (Set<OperatorImpl>) nextOperatorsField.get(wndOpImpl);
+    assertEquals(subsOps.size(), 0);
+  }
+
+  @Test
+  public void testBroadcastChain() throws IllegalAccessException {
+    // test creation of broadcast chain
+    MessageStreamImpl<TestMessageEnvelope> testInput = new MessageStreamImpl<>();
+    TaskContext mockContext = mock(TaskContext.class);
+    testInput.filter(m -> m.getMessage().getEventTime() > 123456L).flatMap(m -> new ArrayList() { { this.add(m); this.add(m); } });
+    testInput.filter(m -> m.getMessage().getEventTime() < 123456L).map(m -> m);
+    RootOperatorImpl operatorChain = OperatorImpls.createOperatorImpls(testInput, mockContext);
+    Set<OperatorImpl> subsSet = (Set<OperatorImpl>) nextOperatorsField.get(operatorChain);
+    assertEquals(subsSet.size(), 2);
+    Iterator<OperatorImpl> iter = subsSet.iterator();
+    // check the first branch w/ flatMap
+    OperatorImpl<TestMessageEnvelope, TestMessageEnvelope> opImpl = iter.next();
+    Set<OperatorImpl> subsOps = (Set<OperatorImpl>) nextOperatorsField.get(opImpl);
+    assertEquals(subsOps.size(), 1);
+    OperatorImpl flatMapImpl = subsOps.iterator().next();
+    subsOps = (Set<OperatorImpl>) nextOperatorsField.get(flatMapImpl);
+    assertEquals(subsOps.size(), 0);
+    // check the second branch w/ map
+    opImpl = iter.next();
+    subsOps = (Set<OperatorImpl>) nextOperatorsField.get(opImpl);
+    assertEquals(subsOps.size(), 1);
+    OperatorImpl mapImpl = subsOps.iterator().next();
+    subsOps = (Set<OperatorImpl>) nextOperatorsField.get(mapImpl);
+    assertEquals(subsOps.size(), 0);
+  }
+
+  @Test
+  public void testJoinChain() throws IllegalAccessException {
+    // test creation of join chain
+    MessageStreamImpl<TestMessageEnvelope> input1 = new MessageStreamImpl<>();
+    MessageStreamImpl<TestMessageEnvelope> input2 = new MessageStreamImpl<>();
+    TaskContext mockContext = mock(TaskContext.class);
+    input1
+        .join(input2, (m1, m2) ->
+            new TestOutputMessageEnvelope(m1.getKey(), m1.getMessage().getValue().length() + m2.getMessage().getValue().length()))
+        .map(m -> m);
+    // now, we create chained operators from each input sources
+    RootOperatorImpl chain1 = OperatorImpls.createOperatorImpls(input1, mockContext);
+    RootOperatorImpl chain2 = OperatorImpls.createOperatorImpls(input2, mockContext);
+    // check that those two chains will merge at map operator
+    // first branch of the join
+    Set<OperatorImpl> subsSet = (Set<OperatorImpl>) nextOperatorsField.get(chain1);
+    assertEquals(subsSet.size(), 1);
+    OperatorImpl<TestMessageEnvelope, TestOutputMessageEnvelope> joinOp1 = subsSet.iterator().next();
+    Set<OperatorImpl> subsOps = (Set<OperatorImpl>) nextOperatorsField.get(joinOp1);
+    assertEquals(subsOps.size(), 1);
+    // the map operator consumes the common join output, where two branches merge
+    OperatorImpl mapImpl = subsOps.iterator().next();
+    // second branch of the join
+    subsSet = (Set<OperatorImpl>) nextOperatorsField.get(chain2);
+    assertEquals(subsSet.size(), 1);
+    OperatorImpl<TestMessageEnvelope, TestOutputMessageEnvelope> joinOp2 = subsSet.iterator().next();
+    assertNotSame(joinOp1, joinOp2);
+    subsOps = (Set<OperatorImpl>) nextOperatorsField.get(joinOp2);
+    assertEquals(subsOps.size(), 1);
+    // make sure that the map operator is the same
+    assertEquals(mapImpl, subsOps.iterator().next());
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/TestProcessorContext.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestProcessorContext.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestProcessorContext.java
deleted file mode 100644
index 224245e..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestProcessorContext.java
+++ /dev/null
@@ -1,40 +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.samza.operators.impl;
-
-import org.apache.samza.operators.TestMessage;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.mock;
-
-
-public class TestProcessorContext {
-  @Test public void testConstructor() {
-    TestMessage mockMsg = mock(TestMessage.class);
-    MessageCollector mockCollector = mock(MessageCollector.class);
-    TaskCoordinator mockTaskCoordinator = mock(TaskCoordinator.class);
-    ProcessorContext<TestMessage> pCntx = new ProcessorContext<>(mockMsg, mockCollector, mockTaskCoordinator);
-    assertEquals(pCntx.getMessage(), mockMsg);
-    assertEquals(pCntx.getCollector(), mockCollector);
-    assertEquals(pCntx.getCoordinator(), mockTaskCoordinator);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSessionWindowImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSessionWindowImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSessionWindowImpl.java
new file mode 100644
index 0000000..c302df7
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSessionWindowImpl.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.samza.operators.impl;
+
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.TestMessageEnvelope;
+import org.apache.samza.operators.spec.WindowOperatorSpec;
+import org.apache.samza.operators.windows.StoreFunctions;
+import org.apache.samza.operators.windows.WindowOutput;
+import org.apache.samza.operators.windows.WindowState;
+import org.apache.samza.storage.kv.Entry;
+import org.apache.samza.storage.kv.KeyValueStore;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentMatcher;
+
+import java.lang.reflect.Field;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.argThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+
+public class TestSessionWindowImpl {
+  Field wndStoreField = null;
+  Field sessWndField = null;
+
+  @Before public void prep() throws NoSuchFieldException {
+    wndStoreField = SessionWindowOperatorImpl.class.getDeclaredField("stateStore");
+    sessWndField = SessionWindowOperatorImpl.class.getDeclaredField("windowSpec");
+    wndStoreField.setAccessible(true);
+    sessWndField.setAccessible(true);
+  }
+
+  @Test
+  public void testConstructor() throws IllegalAccessException, NoSuchFieldException {
+    // test constructing a SessionWindowOperatorImpl w/ expected mock functions
+    WindowOperatorSpec<TestMessageEnvelope, String, WindowState<Integer>, WindowOutput<String, Integer>> wndOp = mock(WindowOperatorSpec.class);
+    SessionWindowOperatorImpl<TestMessageEnvelope, String, WindowState<Integer>, WindowOutput<String, Integer>> sessWnd = new SessionWindowOperatorImpl<>(wndOp);
+    assertEquals(wndOp, sessWndField.get(sessWnd));
+  }
+
+  @Test
+  public void testInitAndProcess() throws IllegalAccessException {
+    WindowOperatorSpec<TestMessageEnvelope, String, WindowState<Integer>, WindowOutput<String, Integer>> wndOp = mock(WindowOperatorSpec.class);
+    BiFunction<TestMessageEnvelope, Entry<String, WindowState<Integer>>, WindowOutput<String, Integer>> mockTxfmFn = mock(BiFunction.class);
+    SessionWindowOperatorImpl<TestMessageEnvelope, String, WindowState<Integer>, WindowOutput<String, Integer>> sessWnd = new SessionWindowOperatorImpl<>(wndOp);
+
+    // construct and init the SessionWindowOperatorImpl object
+    MessageStreamImpl<TestMessageEnvelope> mockInputStrm = mock(MessageStreamImpl.class);
+    StoreFunctions<TestMessageEnvelope, String, WindowState<Integer>> mockStoreFns = mock(StoreFunctions.class);
+    Function<TestMessageEnvelope, String> wndKeyFn = m -> "test-msg-key";
+    when(mockStoreFns.getStoreKeyFn()).thenReturn(wndKeyFn);
+    when(wndOp.getStoreFns()).thenReturn(mockStoreFns);
+    when(wndOp.getStoreName(mockInputStrm)).thenReturn("test-wnd-store");
+    when(wndOp.getTransformFn()).thenReturn(mockTxfmFn);
+    TaskContext mockContext = mock(TaskContext.class);
+    KeyValueStore<String, WindowState<Integer>> mockKvStore = mock(KeyValueStore.class);
+    when(mockContext.getStore("test-wnd-store")).thenReturn(mockKvStore);
+    sessWnd.init(mockInputStrm, mockContext);
+
+    // test onNext() method. Make sure the transformation function and the state update functions are invoked.
+    TestMessageEnvelope mockMsg = mock(TestMessageEnvelope.class);
+    MessageCollector mockCollector = mock(MessageCollector.class);
+    TaskCoordinator mockCoordinator = mock(TaskCoordinator.class);
+    BiFunction<TestMessageEnvelope, WindowState<Integer>, WindowState<Integer>> stateUpdaterFn = mock(BiFunction.class);
+    when(mockStoreFns.getStateUpdaterFn()).thenReturn(stateUpdaterFn);
+    WindowState<Integer> mockNewState = mock(WindowState.class);
+    WindowState<Integer> oldState = mock(WindowState.class);
+    when(mockKvStore.get("test-msg-key")).thenReturn(oldState);
+    when(stateUpdaterFn.apply(mockMsg, oldState)).thenReturn(mockNewState);
+    sessWnd.onNext(mockMsg, mockCollector, mockCoordinator);
+    verify(mockTxfmFn, times(1)).apply(argThat(new ArgumentMatcher<TestMessageEnvelope>() {
+      @Override public boolean matches(Object argument) {
+        TestMessageEnvelope xIn = (TestMessageEnvelope) argument;
+        return xIn.equals(mockMsg);
+      }
+    }), argThat(new ArgumentMatcher<Entry<String, WindowState<Integer>>>() {
+      @Override public boolean matches(Object argument) {
+        Entry<String, WindowState<Integer>> xIn = (Entry<String, WindowState<Integer>>) argument;
+        return xIn.getKey().equals("test-msg-key") && xIn.getValue().equals(oldState);
+      }
+    }));
+    verify(stateUpdaterFn, times(1)).apply(mockMsg, oldState);
+    verify(mockKvStore, times(1)).put("test-msg-key", mockNewState);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSimpleOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSimpleOperatorImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSimpleOperatorImpl.java
deleted file mode 100644
index de029ea..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSimpleOperatorImpl.java
+++ /dev/null
@@ -1,55 +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.samza.operators.impl;
-
-import org.apache.samza.operators.TestMessage;
-import org.apache.samza.operators.TestOutputMessage;
-import org.apache.samza.operators.internal.Operators.StreamOperator;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.function.Function;
-
-import static org.mockito.Mockito.*;
-
-
-public class TestSimpleOperatorImpl {
-
-  @Test public void testSimpleOperator() {
-    StreamOperator<TestMessage, TestOutputMessage> mockOp = mock(StreamOperator.class);
-    Function<TestMessage, Collection<TestOutputMessage>> txfmFn = mock(Function.class);
-    when(mockOp.getFunction()).thenReturn(txfmFn);
-
-    SimpleOperatorImpl<TestMessage, TestOutputMessage> opImpl = spy(new SimpleOperatorImpl<>(mockOp));
-    TestMessage inMsg = mock(TestMessage.class);
-    TestOutputMessage outMsg = mock(TestOutputMessage.class);
-    Collection<TestOutputMessage> mockOutputs = new ArrayList() { {
-        this.add(outMsg);
-      } };
-    when(txfmFn.apply(inMsg)).thenReturn(mockOutputs);
-    MessageCollector mockCollector = mock(MessageCollector.class);
-    TaskCoordinator mockCoordinator = mock(TaskCoordinator.class);
-    opImpl.onNext(inMsg, mockCollector, mockCoordinator);
-    verify(txfmFn, times(1)).apply(inMsg);
-    verify(opImpl, times(1)).nextProcessors(outMsg, mockCollector, mockCoordinator);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
index cdac3fc..ba5b6f8 100644
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
@@ -18,25 +18,28 @@
  */
 package org.apache.samza.operators.impl;
 
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.TestOutputMessage;
-import org.apache.samza.operators.internal.Operators.SinkOperator;
+import org.apache.samza.operators.TestOutputMessageEnvelope;
+import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.operators.spec.SinkOperatorSpec;
 import org.apache.samza.task.MessageCollector;
 import org.apache.samza.task.TaskCoordinator;
 import org.junit.Test;
 
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 
 public class TestSinkOperatorImpl {
 
-  @Test public void testSinkOperator() {
-    SinkOperator<TestOutputMessage> sinkOp = mock(SinkOperator.class);
-    MessageStream.VoidFunction3<TestOutputMessage, MessageCollector, TaskCoordinator> sinkFn = mock(
-        MessageStream.VoidFunction3.class);
-    when(sinkOp.getFunction()).thenReturn(sinkFn);
-    SinkOperatorImpl<TestOutputMessage> sinkImpl = new SinkOperatorImpl<>(sinkOp);
-    TestOutputMessage mockMsg = mock(TestOutputMessage.class);
+  @Test
+  public void testSinkOperator() {
+    SinkOperatorSpec<TestOutputMessageEnvelope> sinkOp = mock(SinkOperatorSpec.class);
+    SinkFunction<TestOutputMessageEnvelope> sinkFn = mock(SinkFunction.class);
+    when(sinkOp.getSinkFn()).thenReturn(sinkFn);
+    SinkOperatorImpl<TestOutputMessageEnvelope> sinkImpl = new SinkOperatorImpl<>(sinkOp);
+    TestOutputMessageEnvelope mockMsg = mock(TestOutputMessageEnvelope.class);
     MessageCollector mockCollector = mock(MessageCollector.class);
     TaskCoordinator mockCoordinator = mock(TaskCoordinator.class);
 

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStateStoreImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStateStoreImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStateStoreImpl.java
deleted file mode 100644
index 5ede757..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStateStoreImpl.java
+++ /dev/null
@@ -1,69 +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.samza.operators.impl;
-
-import org.apache.samza.operators.TestMessage;
-import org.apache.samza.operators.WindowState;
-import org.apache.samza.operators.internal.Operators.StoreFunctions;
-import org.apache.samza.storage.kv.Entry;
-import org.apache.samza.storage.kv.KeyValueStore;
-import org.apache.samza.task.TaskContext;
-import org.junit.Test;
-
-import java.util.function.BiFunction;
-import java.util.function.Function;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.*;
-
-
-public class TestStateStoreImpl {
-  @Test public void testStateStoreImpl() {
-    StoreFunctions<TestMessage, String, WindowState> mockStoreFunctions = mock(StoreFunctions.class);
-    // test constructor
-    StateStoreImpl<TestMessage, String, WindowState> storeImpl = new StateStoreImpl<>(mockStoreFunctions, "myStoreName");
-    TaskContext mockContext = mock(TaskContext.class);
-    KeyValueStore<String, WindowState> mockKvStore = mock(KeyValueStore.class);
-    when(mockContext.getStore("myStoreName")).thenReturn(mockKvStore);
-    // test init()
-    storeImpl.init(mockContext);
-    verify(mockContext, times(1)).getStore("myStoreName");
-    Function<TestMessage, String> wndKeyFn = mock(Function.class);
-    when(mockStoreFunctions.getStoreKeyFinder()).thenReturn(wndKeyFn);
-    TestMessage mockMsg = mock(TestMessage.class);
-    when(wndKeyFn.apply(mockMsg)).thenReturn("myKey");
-    WindowState mockState = mock(WindowState.class);
-    when(mockKvStore.get("myKey")).thenReturn(mockState);
-    // test getState()
-    Entry<String, WindowState> storeEntry = storeImpl.getState(mockMsg);
-    assertEquals(storeEntry.getKey(), "myKey");
-    assertEquals(storeEntry.getValue(), mockState);
-    verify(wndKeyFn, times(1)).apply(mockMsg);
-    verify(mockKvStore, times(1)).get("myKey");
-    Entry<String, WindowState> oldEntry = new Entry<>("myKey", mockState);
-    WindowState mockNewState = mock(WindowState.class);
-    BiFunction<TestMessage, WindowState, WindowState> mockUpdaterFn = mock(BiFunction.class);
-    when(mockStoreFunctions.getStateUpdater()).thenReturn(mockUpdaterFn);
-    when(mockUpdaterFn.apply(mockMsg, mockState)).thenReturn(mockNewState);
-    // test updateState()
-    Entry<String, WindowState> newEntry = storeImpl.updateState(mockMsg, oldEntry);
-    assertEquals(newEntry.getKey(), "myKey");
-    assertEquals(newEntry.getValue(), mockNewState);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java
new file mode 100644
index 0000000..5a3840c
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java
@@ -0,0 +1,60 @@
+/*
+ * 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.samza.operators.impl;
+
+import org.apache.samza.operators.TestMessageEnvelope;
+import org.apache.samza.operators.TestOutputMessageEnvelope;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.spec.StreamOperatorSpec;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+
+public class TestStreamOperatorImpl {
+
+  @Test
+  public void testSimpleOperator() {
+    StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> mockOp = mock(StreamOperatorSpec.class);
+    FlatMapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> txfmFn = mock(FlatMapFunction.class);
+    when(mockOp.getTransformFn()).thenReturn(txfmFn);
+
+    StreamOperatorImpl<TestMessageEnvelope, TestOutputMessageEnvelope> opImpl = spy(new StreamOperatorImpl<>(mockOp));
+    TestMessageEnvelope inMsg = mock(TestMessageEnvelope.class);
+    TestOutputMessageEnvelope outMsg = mock(TestOutputMessageEnvelope.class);
+    Collection<TestOutputMessageEnvelope> mockOutputs = new ArrayList() { {
+        this.add(outMsg);
+      } };
+    when(txfmFn.apply(inMsg)).thenReturn(mockOutputs);
+    MessageCollector mockCollector = mock(MessageCollector.class);
+    TaskCoordinator mockCoordinator = mock(TaskCoordinator.class);
+    opImpl.onNext(inMsg, mockCollector, mockCoordinator);
+    verify(txfmFn, times(1)).apply(inMsg);
+    verify(opImpl, times(1)).propagateResult(outMsg, mockCollector, mockCoordinator);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/window/TestSessionWindowImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/window/TestSessionWindowImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/window/TestSessionWindowImpl.java
deleted file mode 100644
index 719ab99..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/window/TestSessionWindowImpl.java
+++ /dev/null
@@ -1,105 +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.samza.operators.impl.window;
-
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.TestMessage;
-import org.apache.samza.operators.WindowState;
-import org.apache.samza.operators.internal.Operators.StoreFunctions;
-import org.apache.samza.operators.internal.Operators.WindowOperator;
-import org.apache.samza.operators.internal.WindowOutput;
-import org.apache.samza.storage.kv.Entry;
-import org.apache.samza.storage.kv.KeyValueStore;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskContext;
-import org.apache.samza.task.TaskCoordinator;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.ArgumentMatcher;
-
-import java.lang.reflect.Field;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.*;
-
-
-public class TestSessionWindowImpl {
-  Field wndStoreField = null;
-  Field sessWndField = null;
-
-  @Before public void prep() throws NoSuchFieldException {
-    wndStoreField = SessionWindowImpl.class.getDeclaredField("wndStore");
-    sessWndField = SessionWindowImpl.class.getDeclaredField("sessWnd");
-    wndStoreField.setAccessible(true);
-    sessWndField.setAccessible(true);
-  }
-
-  @Test public void testConstructor() throws IllegalAccessException, NoSuchFieldException {
-    // test constructing a SessionWindowImpl w/ expected mock functions
-    WindowOperator<TestMessage, String, WindowState<Integer>, WindowOutput<String, Integer>> wndOp = mock(WindowOperator.class);
-    SessionWindowImpl<TestMessage, String, WindowState<Integer>, WindowOutput<String, Integer>> sessWnd = new SessionWindowImpl<>(wndOp);
-    assertEquals(wndOp, sessWndField.get(sessWnd));
-  }
-
-  @Test public void testInitAndProcess() throws IllegalAccessException {
-    WindowOperator<TestMessage, String, WindowState<Integer>, WindowOutput<String, Integer>> wndOp = mock(WindowOperator.class);
-    BiFunction<TestMessage, Entry<String, WindowState<Integer>>, WindowOutput<String, Integer>> mockTxfmFn = mock(BiFunction.class);
-    SessionWindowImpl<TestMessage, String, WindowState<Integer>, WindowOutput<String, Integer>> sessWnd = new SessionWindowImpl<>(wndOp);
-
-    // construct and init the SessionWindowImpl object
-    MessageStream<TestMessage> mockInputStrm = mock(MessageStream.class);
-    StoreFunctions<TestMessage, String, WindowState<Integer>> mockStoreFns = mock(StoreFunctions.class);
-    Function<TestMessage, String> wndKeyFn = m -> "test-msg-key";
-    when(mockStoreFns.getStoreKeyFinder()).thenReturn(wndKeyFn);
-    when(wndOp.getStoreFunctions()).thenReturn(mockStoreFns);
-    when(wndOp.getStoreName(mockInputStrm)).thenReturn("test-wnd-store");
-    when(wndOp.getFunction()).thenReturn(mockTxfmFn);
-    TaskContext mockContext = mock(TaskContext.class);
-    KeyValueStore<String, WindowState<Integer>> mockKvStore = mock(KeyValueStore.class);
-    when(mockContext.getStore("test-wnd-store")).thenReturn(mockKvStore);
-    sessWnd.init(mockInputStrm, mockContext);
-
-    // test onNext() method. Make sure the transformation function and the state update functions are invoked.
-    TestMessage mockMsg = mock(TestMessage.class);
-    MessageCollector mockCollector = mock(MessageCollector.class);
-    TaskCoordinator mockCoordinator = mock(TaskCoordinator.class);
-    BiFunction<TestMessage, WindowState<Integer>, WindowState<Integer>> stateUpdaterFn = mock(BiFunction.class);
-    when(mockStoreFns.getStateUpdater()).thenReturn(stateUpdaterFn);
-    WindowState<Integer> mockNewState = mock(WindowState.class);
-    WindowState<Integer> oldState = mock(WindowState.class);
-    when(mockKvStore.get("test-msg-key")).thenReturn(oldState);
-    when(stateUpdaterFn.apply(mockMsg, oldState)).thenReturn(mockNewState);
-    sessWnd.onNext(mockMsg, mockCollector, mockCoordinator);
-    verify(mockTxfmFn, times(1)).apply(argThat(new ArgumentMatcher<TestMessage>() {
-      @Override public boolean matches(Object argument) {
-        TestMessage xIn = (TestMessage) argument;
-        return xIn.equals(mockMsg);
-      }
-    }), argThat(new ArgumentMatcher<Entry<String, WindowState<Integer>>>() {
-      @Override public boolean matches(Object argument) {
-        Entry<String, WindowState<Integer>> xIn = (Entry<String, WindowState<Integer>>) argument;
-        return xIn.getKey().equals("test-msg-key") && xIn.getValue().equals(oldState);
-      }
-    }));
-    verify(stateUpdaterFn, times(1)).apply(mockMsg, oldState);
-    verify(mockKvStore, times(1)).put("test-msg-key", mockNewState);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java b/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java
new file mode 100644
index 0000000..028bd67
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java
@@ -0,0 +1,114 @@
+/*
+ * 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.samza.operators.spec;
+
+import org.apache.samza.operators.TestMessageEnvelope;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.windows.StoreFunctions;
+import org.apache.samza.operators.windows.Trigger;
+import org.apache.samza.operators.windows.WindowFn;
+import org.apache.samza.operators.windows.WindowOutput;
+import org.apache.samza.operators.windows.WindowState;
+import org.apache.samza.storage.kv.Entry;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
+public class TestOperatorSpecs {
+  @Test
+  public void testGetStreamOperator() {
+    FlatMapFunction<MessageEnvelope, TestMessageEnvelope> transformFn = m -> new ArrayList<TestMessageEnvelope>() { {
+        this.add(new TestMessageEnvelope(m.getKey().toString(), m.getMessage().toString(), 12345L));
+      } };
+    StreamOperatorSpec<MessageEnvelope, TestMessageEnvelope> strmOp = OperatorSpecs.createStreamOperator(transformFn);
+    assertEquals(strmOp.getTransformFn(), transformFn);
+    assertTrue(strmOp.getOutputStream() instanceof MessageStreamImpl);
+  }
+
+  @Test
+  public void testGetSinkOperator() {
+    SinkFunction<TestMessageEnvelope> sinkFn = (m, c, t) -> { };
+    SinkOperatorSpec<TestMessageEnvelope> sinkOp = OperatorSpecs.createSinkOperator(sinkFn);
+    assertEquals(sinkOp.getSinkFn(), sinkFn);
+    assertTrue(sinkOp.getOutputStream() == null);
+  }
+
+  @Test
+  public void testGetWindowOperator() {
+    WindowFn<TestMessageEnvelope, String, WindowState<Integer>, WindowOutput<String, Integer>> windowFn = mock(WindowFn.class);
+    BiFunction<TestMessageEnvelope, Entry<String, WindowState<Integer>>, WindowOutput<String, Integer>> xFunction = (m, e) -> null;
+    StoreFunctions<TestMessageEnvelope, String, WindowState<Integer>> storeFns = mock(StoreFunctions.class);
+    Trigger<TestMessageEnvelope, WindowState<Integer>> trigger = mock(Trigger.class);
+    MessageStreamImpl<TestMessageEnvelope> mockInput = mock(MessageStreamImpl.class);
+    when(windowFn.getTransformFn()).thenReturn(xFunction);
+    when(windowFn.getStoreFns()).thenReturn(storeFns);
+    when(windowFn.getTrigger()).thenReturn(trigger);
+    when(mockInput.toString()).thenReturn("mockStream1");
+
+    WindowOperatorSpec<TestMessageEnvelope, String, WindowState<Integer>, WindowOutput<String, Integer>> windowOp = OperatorSpecs
+        .createWindowOperator(windowFn);
+    assertEquals(windowOp.getTransformFn(), xFunction);
+    assertEquals(windowOp.getStoreFns(), storeFns);
+    assertEquals(windowOp.getTrigger(), trigger);
+    assertEquals(windowOp.getStoreName(mockInput), String.format("input-mockStream1-wndop-%s", windowOp.toString()));
+  }
+
+  @Test
+  public void testGetPartialJoinOperator() {
+    BiFunction<MessageEnvelope<Object, ?>, MessageEnvelope<Object, ?>, TestMessageEnvelope> merger =
+        (m1, m2) -> new TestMessageEnvelope(m1.getKey().toString(), m2.getMessage().toString(), System.nanoTime());
+    MessageStreamImpl<TestMessageEnvelope> joinOutput = new MessageStreamImpl<>();
+    PartialJoinOperatorSpec<MessageEnvelope<Object, ?>, Object, MessageEnvelope<Object, ?>, TestMessageEnvelope> partialJoin =
+        OperatorSpecs.createPartialJoinOperator(merger, joinOutput);
+
+    assertEquals(partialJoin.getOutputStream(), joinOutput);
+    MessageEnvelope<Object, Object> m = mock(MessageEnvelope.class);
+    MessageEnvelope<Object, Object> s = mock(MessageEnvelope.class);
+    assertEquals(partialJoin.getTransformFn(), merger);
+    assertEquals(partialJoin.getSelfStoreFns().getStoreKeyFn().apply(m), m.getKey());
+    assertEquals(partialJoin.getSelfStoreFns().getStateUpdaterFn().apply(m, s), m);
+    assertEquals(partialJoin.getJoinStoreFns().getStoreKeyFn().apply(m), m.getKey());
+    assertNull(partialJoin.getJoinStoreFns().getStateUpdaterFn());
+  }
+
+  @Test
+  public void testGetMergeOperator() {
+    MessageStreamImpl<TestMessageEnvelope> output = new MessageStreamImpl<>();
+    StreamOperatorSpec<TestMessageEnvelope, TestMessageEnvelope> mergeOp = OperatorSpecs.createMergeOperator(output);
+    Function<TestMessageEnvelope, Collection<TestMessageEnvelope>> mergeFn = t -> new ArrayList<TestMessageEnvelope>() { {
+        this.add(t);
+      } };
+    TestMessageEnvelope t = mock(TestMessageEnvelope.class);
+    assertEquals(mergeOp.getTransformFn().apply(t), mergeFn.apply(t));
+    assertEquals(mergeOp.getOutputStream(), output);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java b/samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java
deleted file mode 100644
index d1b0a88..0000000
--- a/samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java
+++ /dev/null
@@ -1,102 +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.samza.task;
-
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.MessageStreams.SystemMessageStream;
-import org.apache.samza.operators.TriggerBuilder;
-import org.apache.samza.operators.Windows;
-import org.apache.samza.operators.data.IncomingSystemMessage;
-import org.apache.samza.operators.data.Offset;
-import org.apache.samza.operators.task.StreamOperatorTask;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.util.Collection;
-
-
-/**
- * Example implementation of split stream tasks
- *
- */
-public class BroadcastOperatorTask implements StreamOperatorTask {
-  class MessageType {
-    String field1;
-    String field2;
-    String field3;
-    String field4;
-    String parKey;
-    private long timestamp;
-
-    public long getTimestamp() {
-      return this.timestamp;
-    }
-  }
-
-  class JsonMessage extends InputJsonSystemMessage<MessageType> {
-
-    JsonMessage(String key, MessageType data, Offset offset, long timestamp, SystemStreamPartition partition) {
-      super(key, data, offset, timestamp, partition);
-    }
-  }
-
-  @Override public void initOperators(Collection<SystemMessageStream> sources) {
-    sources.forEach(source -> {
-        MessageStream<JsonMessage> inputStream = source.map(this::getInputMessage);
-
-        inputStream.filter(this::myFilter1).
-          window(Windows.<JsonMessage, String>intoSessionCounter(
-              m -> String.format("%s-%s", m.getMessage().field1, m.getMessage().field2)).
-            setTriggers(TriggerBuilder.<JsonMessage, Integer>earlyTriggerWhenExceedWndLen(100).
-              addLateTriggerOnSizeLimit(10).
-              addTimeoutSinceLastMessage(30000)));
-
-        inputStream.filter(this::myFilter2).
-          window(Windows.<JsonMessage, String>intoSessions(
-              m -> String.format("%s-%s", m.getMessage().field3, m.getMessage().field4)).
-            setTriggers(TriggerBuilder.<JsonMessage, Collection<JsonMessage>>earlyTriggerWhenExceedWndLen(100).
-              addTimeoutSinceLastMessage(30000)));
-
-        inputStream.filter(this::myFilter3).
-          window(Windows.<JsonMessage, String, MessageType>intoSessions(
-              m -> String.format("%s-%s", m.getMessage().field3, m.getMessage().field4), m -> m.getMessage()).
-            setTriggers(TriggerBuilder.<JsonMessage, Collection<MessageType>>earlyTriggerOnEventTime(m -> m.getTimestamp(), 30000).
-              addTimeoutSinceFirstMessage(60000)));
-      }
-    );
-  }
-
-  JsonMessage getInputMessage(IncomingSystemMessage m1) {
-    return (JsonMessage) m1.getMessage();
-  }
-
-  boolean myFilter1(JsonMessage m1) {
-    // Do user defined processing here
-    return m1.getMessage().parKey.equals("key1");
-  }
-
-  boolean myFilter2(JsonMessage m1) {
-    // Do user defined processing here
-    return m1.getMessage().parKey.equals("key2");
-  }
-
-  boolean myFilter3(JsonMessage m1) {
-    return m1.getMessage().parKey.equals("key3");
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/task/InputJsonSystemMessage.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/task/InputJsonSystemMessage.java b/samza-operator/src/test/java/org/apache/samza/task/InputJsonSystemMessage.java
deleted file mode 100644
index 88aa159..0000000
--- a/samza-operator/src/test/java/org/apache/samza/task/InputJsonSystemMessage.java
+++ /dev/null
@@ -1,67 +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.samza.task;
-
-import org.apache.samza.operators.data.InputSystemMessage;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.data.Offset;
-import org.apache.samza.system.SystemStreamPartition;
-
-
-/**
- * Example input message w/ Json message body and string as the key.
- */
-
-public class InputJsonSystemMessage<T> implements Message<String, T>, InputSystemMessage<Offset> {
-
-  private final String key;
-  private final T data;
-  private final Offset offset;
-  private final long timestamp;
-  private final SystemStreamPartition partition;
-
-  InputJsonSystemMessage(String key, T data, Offset offset, long timestamp, SystemStreamPartition partition) {
-    this.key = key;
-    this.data = data;
-    this.offset = offset;
-    this.timestamp = timestamp;
-    this.partition = partition;
-  }
-
-  @Override public T getMessage() {
-    return this.data;
-  }
-
-  @Override public String getKey() {
-    return this.key;
-  }
-
-  @Override public long getTimestamp() {
-    return this.timestamp;
-  }
-
-  @Override public Offset getOffset() {
-    return this.offset;
-  }
-
-  @Override public SystemStreamPartition getSystemStreamPartition() {
-    return this.partition;
-  }
-}
-


[4/6] samza git commit: SAMZA-1054: Refactor Operator APIs

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/TestOutputMessage.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/TestOutputMessage.java b/samza-api/src/test/java/org/apache/samza/operators/TestOutputMessage.java
deleted file mode 100644
index 14e6562..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/TestOutputMessage.java
+++ /dev/null
@@ -1,47 +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.samza.operators;
-
-import org.apache.samza.operators.data.Message;
-
-
-public class TestOutputMessage implements Message<String, Integer> {
-  private final String key;
-  private final Integer value;
-  private final long timestamp;
-
-  public TestOutputMessage(String key, Integer value, long timestamp) {
-    this.key = key;
-    this.value = value;
-    this.timestamp = timestamp;
-  }
-
-  @Override public Integer getMessage() {
-    return this.value;
-  }
-
-  @Override public String getKey() {
-    return this.key;
-  }
-
-  @Override public long getTimestamp() {
-    return this.timestamp;
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/TestOutputMessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/TestOutputMessageEnvelope.java b/samza-api/src/test/java/org/apache/samza/operators/TestOutputMessageEnvelope.java
new file mode 100644
index 0000000..284b30b
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/operators/TestOutputMessageEnvelope.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.samza.operators;
+
+import org.apache.samza.operators.data.MessageEnvelope;
+
+
+public class TestOutputMessageEnvelope implements MessageEnvelope<String, Integer> {
+  private final String key;
+  private final Integer value;
+
+  public TestOutputMessageEnvelope(String key, Integer value) {
+    this.key = key;
+    this.value = value;
+  }
+
+  @Override
+  public Integer getMessage() {
+    return this.value;
+  }
+
+  @Override
+  public String getKey() {
+    return this.key;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/TestTriggerBuilder.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/TestTriggerBuilder.java b/samza-api/src/test/java/org/apache/samza/operators/TestTriggerBuilder.java
deleted file mode 100644
index 927b14b..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/TestTriggerBuilder.java
+++ /dev/null
@@ -1,214 +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.samza.operators;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import java.lang.reflect.Field;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-
-public class TestTriggerBuilder {
-  private Field earlyTriggerField;
-  private Field lateTriggerField;
-  private Field timerTriggerField;
-  private Field earlyTriggerUpdater;
-  private Field lateTriggerUpdater;
-
-  @Before
-  public void testPrep() throws Exception {
-    this.earlyTriggerField = TriggerBuilder.class.getDeclaredField("earlyTrigger");
-    this.lateTriggerField = TriggerBuilder.class.getDeclaredField("lateTrigger");
-    this.timerTriggerField = TriggerBuilder.class.getDeclaredField("timerTrigger");
-    this.earlyTriggerUpdater = TriggerBuilder.class.getDeclaredField("earlyTriggerUpdater");
-    this.lateTriggerUpdater = TriggerBuilder.class.getDeclaredField("lateTriggerUpdater");
-
-    this.earlyTriggerField.setAccessible(true);
-    this.lateTriggerField.setAccessible(true);
-    this.timerTriggerField.setAccessible(true);
-    this.earlyTriggerUpdater.setAccessible(true);
-    this.lateTriggerUpdater.setAccessible(true);
-  }
-
-  @Test public void testStaticCreators() throws NoSuchFieldException, IllegalAccessException {
-    TriggerBuilder<TestMessage, Collection<TestMessage>> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
-    BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean> triggerField =
-        (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
-    WindowState<Collection<TestMessage>> mockState = mock(WindowState.class);
-    when(mockState.getNumberMessages()).thenReturn(200L);
-    assertFalse(triggerField.apply(null, mockState));
-    when(mockState.getNumberMessages()).thenReturn(2000L);
-    assertTrue(triggerField.apply(null, mockState));
-
-    Function<TestMessage, Boolean> tokenFunc = m -> true;
-    builder = TriggerBuilder.earlyTriggerOnTokenMsg(tokenFunc);
-    triggerField = (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
-    TestMessage m = mock(TestMessage.class);
-    assertTrue(triggerField.apply(m, mockState));
-
-    builder = TriggerBuilder.earlyTriggerOnEventTime(TestMessage::getTimestamp, 30000L);
-    triggerField = (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
-    when(mockState.getEarliestEventTimeNs()).thenReturn(1000000000L);
-    when(mockState.getLatestEventTimeNs()).thenReturn(20000000000L);
-    when(m.getTimestamp()).thenReturn(19999000000L);
-    assertFalse(triggerField.apply(m, mockState));
-    when(m.getTimestamp()).thenReturn(32000000000L);
-    assertTrue(triggerField.apply(m, mockState));
-    when(m.getTimestamp()).thenReturn(1001000000L);
-    when(mockState.getLatestEventTimeNs()).thenReturn(32000000000L);
-    assertTrue(triggerField.apply(m, mockState));
-
-    BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean> mockFunc = mock(BiFunction.class);
-    builder = TriggerBuilder.earlyTrigger(mockFunc);
-    triggerField = (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
-    assertEquals(triggerField, mockFunc);
-
-    builder = TriggerBuilder.timeoutSinceFirstMessage(10000L);
-    Function<WindowState<Collection<TestMessage>>, Boolean> timerTrigger =
-        (Function<WindowState<Collection<TestMessage>>, Boolean>) this.timerTriggerField.get(builder);
-    when(mockState.getFirstMessageTimeNs()).thenReturn(0L);
-    assertTrue(timerTrigger.apply(mockState));
-    // set the firstMessageTimeNs to 9 second earlier, giving the test 1 second to fire up the timerTrigger before assertion
-    when(mockState.getFirstMessageTimeNs()).thenReturn(TimeUnit.MILLISECONDS.toNanos(System.currentTimeMillis() - 9000L));
-    assertFalse(timerTrigger.apply(mockState));
-
-    builder = TriggerBuilder.timeoutSinceLastMessage(10000L);
-    timerTrigger = (Function<WindowState<Collection<TestMessage>>, Boolean>) this.timerTriggerField.get(builder);
-    when(mockState.getLastMessageTimeNs()).thenReturn(0L);
-    assertTrue(timerTrigger.apply(mockState));
-    // set the lastMessageTimeNs to 9 second earlier, giving the test 1 second to fire up the timerTrigger before assertion
-    when(mockState.getLastMessageTimeNs()).thenReturn(TimeUnit.MILLISECONDS.toNanos(System.currentTimeMillis() - 9000));
-    assertFalse(timerTrigger.apply(mockState));
-  }
-
-  @Test public void testAddTimerTriggers() throws IllegalAccessException {
-    TriggerBuilder<TestMessage, Collection<TestMessage>> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
-    builder.addTimeoutSinceFirstMessage(10000L);
-    // exam that both earlyTrigger and timer triggers are set up
-    BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean> triggerField =
-        (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
-    WindowState<Collection<TestMessage>> mockState = mock(WindowState.class);
-    when(mockState.getNumberMessages()).thenReturn(200L);
-    assertFalse(triggerField.apply(null, mockState));
-    // check the timer trigger
-    Function<WindowState<Collection<TestMessage>>, Boolean> timerTrigger =
-        (Function<WindowState<Collection<TestMessage>>, Boolean>) this.timerTriggerField.get(builder);
-    when(mockState.getFirstMessageTimeNs()).thenReturn(0L);
-    assertTrue(timerTrigger.apply(mockState));
-    // set the firstMessageTimeNs to 9 second earlier, giving the test 1 second to fire up the timerTrigger before assertion
-    when(mockState.getFirstMessageTimeNs()).thenReturn(TimeUnit.MILLISECONDS.toNanos(System.currentTimeMillis() - 9000L));
-    assertFalse(timerTrigger.apply(mockState));
-
-    // exam that both early trigger and timer triggers are set up
-    builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
-    triggerField = (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
-    mockState = mock(WindowState.class);
-    when(mockState.getNumberMessages()).thenReturn(200L);
-    assertFalse(triggerField.apply(null, mockState));
-    builder.addTimeoutSinceLastMessage(20000L);
-    // check the timer trigger
-    timerTrigger = (Function<WindowState<Collection<TestMessage>>, Boolean>) this.timerTriggerField.get(builder);
-    when(mockState.getLastMessageTimeNs()).thenReturn(0L);
-    assertTrue(timerTrigger.apply(mockState));
-    // set the firstMessageTimeNs to 9 second earlier, giving the test 1 second to fire up the timerTrigger before assertion
-    when(mockState.getLastMessageTimeNs()).thenReturn(TimeUnit.MILLISECONDS.toNanos(System.currentTimeMillis() - 9000L));
-    assertFalse(timerTrigger.apply(mockState));
-  }
-
-  @Test public void testAddLateTriggers() throws IllegalAccessException {
-    TriggerBuilder<TestMessage, Collection<TestMessage>> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
-    builder.addLateTriggerOnSizeLimit(10000L);
-    // exam that both earlyTrigger and lateTriggers are set up
-    BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean> earlyTrigger =
-        (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
-    WindowState<Collection<TestMessage>> mockState = mock(WindowState.class);
-    when(mockState.getNumberMessages()).thenReturn(200L);
-    assertFalse(earlyTrigger.apply(null, mockState));
-    // check the late trigger
-    BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean> lateTrigger =
-        (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.lateTriggerField.get(builder);
-    assertFalse(lateTrigger.apply(null, mockState));
-    // set the number of messages to 10001 to trigger the late trigger
-    when(mockState.getNumberMessages()).thenReturn(10001L);
-    assertTrue(lateTrigger.apply(null, mockState));
-
-    builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
-    builder.addLateTrigger((m, s) -> s.getOutputValue().size() > 0);
-    // exam that both earlyTrigger and lateTriggers are set up
-    earlyTrigger = (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
-    mockState = mock(WindowState.class);
-    when(mockState.getNumberMessages()).thenReturn(200L);
-    assertFalse(earlyTrigger.apply(null, mockState));
-    // exam the lateTrigger
-    when(mockState.getOutputValue()).thenReturn(new ArrayList<>());
-    lateTrigger = (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.lateTriggerField.get(builder);
-    assertFalse(lateTrigger.apply(null, mockState));
-    List<TestMessage> mockList = mock(ArrayList.class);
-    when(mockList.size()).thenReturn(200);
-    when(mockState.getOutputValue()).thenReturn(mockList);
-    assertTrue(lateTrigger.apply(null, mockState));
-  }
-
-  @Test public void testAddTriggerUpdater() throws IllegalAccessException {
-    TriggerBuilder<TestMessage, Collection<TestMessage>> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
-    builder.onEarlyTrigger(c -> {
-        c.clear();
-        return c;
-      });
-    List<TestMessage> collection = new ArrayList<TestMessage>() { {
-        for (int i = 0; i < 10; i++) {
-          this.add(new TestMessage(String.format("key-%d", i), "string-value", System.nanoTime()));
-        }
-      } };
-    // exam that earlyTriggerUpdater is set up
-    Function<WindowState<Collection<TestMessage>>, WindowState<Collection<TestMessage>>> earlyTriggerUpdater =
-        (Function<WindowState<Collection<TestMessage>>, WindowState<Collection<TestMessage>>>) this.earlyTriggerUpdater.get(builder);
-    WindowState<Collection<TestMessage>> mockState = mock(WindowState.class);
-    when(mockState.getOutputValue()).thenReturn(collection);
-    earlyTriggerUpdater.apply(mockState);
-    assertTrue(collection.isEmpty());
-
-    collection.add(new TestMessage("key-to-stay", "string-to-stay", System.nanoTime()));
-    collection.add(new TestMessage("key-to-remove", "string-to-remove", System.nanoTime()));
-    builder.onLateTrigger(c -> {
-        c.removeIf(t -> t.getKey().equals("key-to-remove"));
-        return c;
-      });
-    // check the late trigger updater
-    Function<WindowState<Collection<TestMessage>>, WindowState<Collection<TestMessage>>> lateTriggerUpdater =
-        (Function<WindowState<Collection<TestMessage>>, WindowState<Collection<TestMessage>>>) this.lateTriggerUpdater.get(builder);
-    when(mockState.getOutputValue()).thenReturn(collection);
-    lateTriggerUpdater.apply(mockState);
-    assertTrue(collection.size() == 1);
-    assertFalse(collection.get(0).isDelete());
-    assertEquals(collection.get(0).getKey(), "key-to-stay");
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/TestWindows.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/TestWindows.java b/samza-api/src/test/java/org/apache/samza/operators/TestWindows.java
deleted file mode 100644
index 8a25a96..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/TestWindows.java
+++ /dev/null
@@ -1,106 +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.samza.operators;
-
-import org.apache.samza.operators.Windows.Window;
-import org.apache.samza.operators.internal.Trigger;
-import org.apache.samza.operators.internal.WindowOutput;
-import org.junit.Test;
-
-import java.lang.reflect.Field;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-
-public class TestWindows {
-
-  @Test public void testSessionWindows() throws NoSuchFieldException, IllegalAccessException {
-    // test constructing the default session window
-    Window<TestMessage, String, Collection<TestMessage>, WindowOutput<String, Collection<TestMessage>>> testWnd = Windows.intoSessions(
-        TestMessage::getKey);
-    assertTrue(testWnd instanceof Windows.SessionWindow);
-    Field wndKeyFuncField = Windows.SessionWindow.class.getDeclaredField("wndKeyFunction");
-    Field aggregatorField = Windows.SessionWindow.class.getDeclaredField("aggregator");
-    wndKeyFuncField.setAccessible(true);
-    aggregatorField.setAccessible(true);
-    Function<TestMessage, String> wndKeyFunc = (Function<TestMessage, String>) wndKeyFuncField.get(testWnd);
-    assertEquals(wndKeyFunc.apply(new TestMessage("test-key", "test-value", 0)), "test-key");
-    BiFunction<TestMessage, Collection<TestMessage>, Collection<TestMessage>> aggrFunc =
-        (BiFunction<TestMessage, Collection<TestMessage>, Collection<TestMessage>>) aggregatorField.get(testWnd);
-    TestMessage mockMsg = mock(TestMessage.class);
-    Collection<TestMessage> collection = aggrFunc.apply(mockMsg, new ArrayList<>());
-    assertTrue(collection.size() == 1);
-    assertTrue(collection.contains(mockMsg));
-
-    // test constructing the session window w/ customized session info
-    Window<TestMessage, String, Collection<Character>, WindowOutput<String, Collection<Character>>> testWnd2 = Windows.intoSessions(
-        m -> String.format("key-%d", m.getTimestamp()), m -> m.getMessage().charAt(0));
-    assertTrue(testWnd2 instanceof Windows.SessionWindow);
-    wndKeyFunc = (Function<TestMessage, String>) wndKeyFuncField.get(testWnd2);
-    aggrFunc = (BiFunction<TestMessage, Collection<TestMessage>, Collection<TestMessage>>) aggregatorField.get(testWnd2);
-    assertEquals(wndKeyFunc.apply(new TestMessage("test-key", "test-value", 0)), "key-0");
-    when(mockMsg.getMessage()).thenReturn("x-001");
-    collection = aggrFunc.apply(mockMsg, new ArrayList<>());
-    assertTrue(collection.size() == 1);
-    assertTrue(collection.contains('x'));
-
-    // test constructing session window w/ a default counter
-    Window<TestMessage, String, Integer, WindowOutput<String, Integer>> testCounter = Windows.intoSessionCounter(
-        m -> String.format("key-%d", m.getTimestamp()));
-    assertTrue(testCounter instanceof Windows.SessionWindow);
-    wndKeyFunc = (Function<TestMessage, String>) wndKeyFuncField.get(testCounter);
-    BiFunction<TestMessage, Integer, Integer> counterFn = (BiFunction<TestMessage, Integer, Integer>) aggregatorField.get(testCounter);
-    when(mockMsg.getTimestamp()).thenReturn(12345L);
-    assertEquals(wndKeyFunc.apply(mockMsg), "key-12345");
-    assertEquals(counterFn.apply(mockMsg, 1), Integer.valueOf(2));
-  }
-
-  @Test public void testSetTriggers() throws NoSuchFieldException, IllegalAccessException {
-    Window<TestMessage, String, Integer, WindowOutput<String, Integer>> testCounter = Windows.intoSessionCounter(
-        m -> String.format("key-%d", m.getTimestamp()));
-    // test session window w/ a trigger
-    TriggerBuilder<TestMessage, Integer> triggerBuilder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000L);
-    testCounter.setTriggers(triggerBuilder);
-    Trigger<TestMessage, WindowState<Integer>> expectedTrigger = triggerBuilder.build();
-    Trigger<TestMessage, WindowState<Integer>> actualTrigger = Windows.getInternalWindowFn(testCounter).getTrigger();
-    // examine all trigger fields are expected
-    Field earlyTriggerField = Trigger.class.getDeclaredField("earlyTrigger");
-    Field lateTriggerField = Trigger.class.getDeclaredField("lateTrigger");
-    Field timerTriggerField = Trigger.class.getDeclaredField("timerTrigger");
-    Field earlyTriggerUpdater = Trigger.class.getDeclaredField("earlyTriggerUpdater");
-    Field lateTriggerUpdater = Trigger.class.getDeclaredField("lateTriggerUpdater");
-    earlyTriggerField.setAccessible(true);
-    lateTriggerField.setAccessible(true);
-    timerTriggerField.setAccessible(true);
-    earlyTriggerUpdater.setAccessible(true);
-    lateTriggerUpdater.setAccessible(true);
-    assertEquals(earlyTriggerField.get(expectedTrigger), earlyTriggerField.get(actualTrigger));
-    assertEquals(lateTriggerField.get(expectedTrigger), lateTriggerField.get(actualTrigger));
-    assertEquals(timerTriggerField.get(expectedTrigger), timerTriggerField.get(actualTrigger));
-    assertEquals(earlyTriggerUpdater.get(expectedTrigger), earlyTriggerUpdater.get(actualTrigger));
-    assertEquals(lateTriggerUpdater.get(expectedTrigger), lateTriggerUpdater.get(actualTrigger));
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/data/TestIncomingSystemMessage.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/data/TestIncomingSystemMessage.java b/samza-api/src/test/java/org/apache/samza/operators/data/TestIncomingSystemMessage.java
index b734e87..7bd62a7 100644
--- a/samza-api/src/test/java/org/apache/samza/operators/data/TestIncomingSystemMessage.java
+++ b/samza-api/src/test/java/org/apache/samza/operators/data/TestIncomingSystemMessage.java
@@ -30,9 +30,10 @@ import static org.mockito.Mockito.when;
 
 public class TestIncomingSystemMessage {
 
-  @Test public void testConstructor() {
+  @Test
+  public void testConstructor() {
     IncomingMessageEnvelope ime = mock(IncomingMessageEnvelope.class);
-    IncomingSystemMessage ism = new IncomingSystemMessage(ime);
+    IncomingSystemMessageEnvelope ism = new IncomingSystemMessageEnvelope(ime);
 
     Object mockKey = mock(Object.class);
     Object mockValue = mock(Object.class);

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/data/TestLongOffset.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/data/TestLongOffset.java b/samza-api/src/test/java/org/apache/samza/operators/data/TestLongOffset.java
index 943c47f..7838896 100644
--- a/samza-api/src/test/java/org/apache/samza/operators/data/TestLongOffset.java
+++ b/samza-api/src/test/java/org/apache/samza/operators/data/TestLongOffset.java
@@ -28,7 +28,8 @@ import static org.mockito.Mockito.mock;
 
 public class TestLongOffset {
 
-  @Test public void testConstructor() throws Exception {
+  @Test
+  public void testConstructor() throws Exception {
     LongOffset o1 = new LongOffset("12345");
     Field offsetField = LongOffset.class.getDeclaredField("offset");
     offsetField.setAccessible(true);
@@ -47,7 +48,8 @@ public class TestLongOffset {
     }
   }
 
-  @Test public void testComparator() {
+  @Test
+  public void testComparator() {
     LongOffset o1 = new LongOffset("11111");
     Offset other = mock(Offset.class);
     try {
@@ -65,7 +67,8 @@ public class TestLongOffset {
     assertEquals(o1.compareTo(o4), 0);
   }
 
-  @Test public void testEquals() {
+  @Test
+  public void testEquals() {
     LongOffset o1 = new LongOffset("12345");
     Offset other = mock(Offset.class);
     assertFalse(o1.equals(other));

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/internal/TestOperators.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/internal/TestOperators.java b/samza-api/src/test/java/org/apache/samza/operators/internal/TestOperators.java
deleted file mode 100644
index d994486..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/internal/TestOperators.java
+++ /dev/null
@@ -1,128 +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.samza.operators.internal;
-
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.WindowState;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.storage.kv.Entry;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-
-public class TestOperators {
-
-  private class TestMessage implements Message<String, Object> {
-    private final long timestamp;
-    private final String key;
-    private final Object msg;
-
-
-    TestMessage(String key, Object msg, long timestamp) {
-      this.timestamp = timestamp;
-      this.key = key;
-      this.msg = msg;
-    }
-
-    @Override public Object getMessage() {
-      return this.msg;
-    }
-
-    @Override public String getKey() {
-      return this.key;
-    }
-
-    @Override public long getTimestamp() {
-      return this.timestamp;
-    }
-  }
-
-  @Test public void testGetStreamOperator() {
-    Function<Message, Collection<TestMessage>> transformFn = m -> new ArrayList<TestMessage>() { {
-        this.add(new TestMessage(m.getKey().toString(), m.getMessage(), 12345L));
-      } };
-    Operators.StreamOperator<Message, TestMessage> strmOp = Operators.getStreamOperator(transformFn);
-    assertEquals(strmOp.getFunction(), transformFn);
-    assertTrue(strmOp.getOutputStream() instanceof MessageStream);
-  }
-
-  @Test public void testGetSinkOperator() {
-    MessageStream.VoidFunction3<TestMessage, MessageCollector, TaskCoordinator> sinkFn = (m, c, t) -> { };
-    Operators.SinkOperator<TestMessage> sinkOp = Operators.getSinkOperator(sinkFn);
-    assertEquals(sinkOp.getFunction(), sinkFn);
-    assertTrue(sinkOp.getOutputStream() == null);
-  }
-
-  @Test public void testGetWindowOperator() {
-    WindowFn<TestMessage, String, WindowState<Integer>, WindowOutput<String, Integer>> windowFn = mock(WindowFn.class);
-    BiFunction<TestMessage, Entry<String, WindowState<Integer>>, WindowOutput<String, Integer>> xFunction = (m, e) -> null;
-    Operators.StoreFunctions<TestMessage, String, WindowState<Integer>> storeFns = mock(Operators.StoreFunctions.class);
-    Trigger<TestMessage, WindowState<Integer>> trigger = mock(Trigger.class);
-    MessageStream<TestMessage> mockInput = mock(MessageStream.class);
-    when(windowFn.getTransformFunc()).thenReturn(xFunction);
-    when(windowFn.getStoreFuncs()).thenReturn(storeFns);
-    when(windowFn.getTrigger()).thenReturn(trigger);
-    when(mockInput.toString()).thenReturn("mockStream1");
-
-    Operators.WindowOperator<TestMessage, String, WindowState<Integer>, WindowOutput<String, Integer>> windowOp = Operators.getWindowOperator(windowFn);
-    assertEquals(windowOp.getFunction(), xFunction);
-    assertEquals(windowOp.getStoreFunctions(), storeFns);
-    assertEquals(windowOp.getTrigger(), trigger);
-    assertEquals(windowOp.getStoreName(mockInput), String.format("input-mockStream1-wndop-%s", windowOp.toString()));
-  }
-
-  @Test public void testGetPartialJoinOperator() {
-    BiFunction<Message<Object, ?>, Message<Object, ?>, TestMessage> merger =
-        (m1, m2) -> new TestMessage(m1.getKey().toString(), m2.getMessage(),
-            Math.max(m1.getTimestamp(), m2.getTimestamp()));
-    MessageStream<TestMessage> joinOutput = new MessageStream<>();
-    Operators.PartialJoinOperator<Message<Object, ?>, Object, Message<Object, ?>, TestMessage> partialJoin =
-        Operators.getPartialJoinOperator(merger, joinOutput);
-
-    assertEquals(partialJoin.getOutputStream(), joinOutput);
-    Message<Object, Object> m = mock(Message.class);
-    Message<Object, Object> s = mock(Message.class);
-    assertEquals(partialJoin.getFunction(), merger);
-    assertEquals(partialJoin.getSelfStoreFunctions().getStoreKeyFinder().apply(m), m.getKey());
-    assertEquals(partialJoin.getSelfStoreFunctions().getStateUpdater().apply(m, s), m);
-    assertEquals(partialJoin.getJoinStoreFunctions().getStoreKeyFinder().apply(m), m.getKey());
-    assertNull(partialJoin.getJoinStoreFunctions().getStateUpdater());
-  }
-
-  @Test public void testGetMergeOperator() {
-    MessageStream<TestMessage> output = new MessageStream<>();
-    Operators.StreamOperator<TestMessage, TestMessage> mergeOp = Operators.getMergeOperator(output);
-    Function<TestMessage, Collection<TestMessage>> mergeFn = t -> new ArrayList<TestMessage>() { {
-        this.add(t);
-      } };
-    TestMessage t = mock(TestMessage.class);
-    assertEquals(mergeOp.getFunction().apply(t), mergeFn.apply(t));
-    assertEquals(mergeOp.getOutputStream(), output);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/internal/TestTrigger.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/internal/TestTrigger.java b/samza-api/src/test/java/org/apache/samza/operators/internal/TestTrigger.java
deleted file mode 100644
index 0f35a7c..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/internal/TestTrigger.java
+++ /dev/null
@@ -1,68 +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.samza.operators.internal;
-
-import org.apache.samza.operators.WindowState;
-import org.apache.samza.operators.data.Message;
-import org.junit.Test;
-
-import java.lang.reflect.Field;
-import java.util.concurrent.TimeUnit;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-
-import static org.junit.Assert.assertEquals;
-
-
-public class TestTrigger {
-
-  @Test public void testConstructor() throws Exception {
-    BiFunction<Message<Object, Object>, WindowState<Integer>, Boolean> earlyTrigger = (m, s) -> s.getOutputValue() > 1000;
-    BiFunction<Message<Object, Object>, WindowState<Integer>, Boolean> lateTrigger = (m, s) -> s.getOutputValue() > 1000;
-    Function<WindowState<Integer>, Boolean> timerTrigger = s -> TimeUnit.NANOSECONDS.toMillis(s.getLastMessageTimeNs()) + 50000 < System.currentTimeMillis();
-    Function<WindowState<Integer>, WindowState<Integer>> earlyTriggerUpdater = s -> {
-      s.setOutputValue(0);
-      return s;
-    };
-    Function<WindowState<Integer>, WindowState<Integer>> lateTriggerUpdater = s -> {
-      s.setOutputValue(1);
-      return s;
-    };
-
-    Trigger<Message<Object, Object>, WindowState<Integer>> trigger = Trigger.createTrigger(timerTrigger, earlyTrigger, lateTrigger,
-        earlyTriggerUpdater, lateTriggerUpdater);
-
-    Field earlyTriggerField = Trigger.class.getDeclaredField("earlyTrigger");
-    Field timerTriggerField = Trigger.class.getDeclaredField("timerTrigger");
-    Field lateTriggerField = Trigger.class.getDeclaredField("lateTrigger");
-    Field earlyTriggerUpdaterField = Trigger.class.getDeclaredField("earlyTriggerUpdater");
-    Field lateTriggerUpdaterField = Trigger.class.getDeclaredField("lateTriggerUpdater");
-    earlyTriggerField.setAccessible(true);
-    lateTriggerField.setAccessible(true);
-    timerTriggerField.setAccessible(true);
-    earlyTriggerUpdaterField.setAccessible(true);
-    lateTriggerUpdaterField.setAccessible(true);
-
-    assertEquals(earlyTrigger, earlyTriggerField.get(trigger));
-    assertEquals(timerTrigger, timerTriggerField.get(trigger));
-    assertEquals(lateTrigger, lateTriggerField.get(trigger));
-    assertEquals(earlyTriggerUpdater, earlyTriggerUpdaterField.get(trigger));
-    assertEquals(lateTriggerUpdater, lateTriggerUpdaterField.get(trigger));
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/internal/TestWindowOutput.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/internal/TestWindowOutput.java b/samza-api/src/test/java/org/apache/samza/operators/internal/TestWindowOutput.java
deleted file mode 100644
index 268c9fc..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/internal/TestWindowOutput.java
+++ /dev/null
@@ -1,36 +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.samza.operators.internal;
-
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-
-
-public class TestWindowOutput {
-
-  @Test public void testConstructor() {
-    WindowOutput<String, Integer> wndOutput = WindowOutput.of("testMsg", 10);
-    assertEquals(wndOutput.getKey(), "testMsg");
-    assertEquals(wndOutput.getMessage(), Integer.valueOf(10));
-    assertFalse(wndOutput.isDelete());
-    assertEquals(wndOutput.getTimestamp(), 0);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/windows/TestTrigger.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/windows/TestTrigger.java b/samza-api/src/test/java/org/apache/samza/operators/windows/TestTrigger.java
new file mode 100644
index 0000000..a91af24
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/operators/windows/TestTrigger.java
@@ -0,0 +1,68 @@
+/*
+ * 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.samza.operators.windows;
+
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertEquals;
+
+
+public class TestTrigger {
+
+  @Test
+  public void testConstructor() throws Exception {
+    BiFunction<MessageEnvelope<Object, Object>, WindowState<Integer>, Boolean> earlyTrigger = (m, s) -> s.getOutputValue() > 1000;
+    BiFunction<MessageEnvelope<Object, Object>, WindowState<Integer>, Boolean> lateTrigger = (m, s) -> s.getOutputValue() > 1000;
+    Function<WindowState<Integer>, Boolean> timerTrigger = s -> TimeUnit.NANOSECONDS.toMillis(s.getLastMessageTimeNs()) + 50000 < System.currentTimeMillis();
+    Function<WindowState<Integer>, WindowState<Integer>> earlyTriggerUpdater = s -> {
+      s.setOutputValue(0);
+      return s;
+    };
+    Function<WindowState<Integer>, WindowState<Integer>> lateTriggerUpdater = s -> {
+      s.setOutputValue(1);
+      return s;
+    };
+
+    Trigger<MessageEnvelope<Object, Object>, WindowState<Integer>> trigger = Trigger.createTrigger(timerTrigger, earlyTrigger, lateTrigger,
+        earlyTriggerUpdater, lateTriggerUpdater);
+
+    Field earlyTriggerField = Trigger.class.getDeclaredField("earlyTrigger");
+    Field timerTriggerField = Trigger.class.getDeclaredField("timerTrigger");
+    Field lateTriggerField = Trigger.class.getDeclaredField("lateTrigger");
+    Field earlyTriggerUpdaterField = Trigger.class.getDeclaredField("earlyTriggerUpdater");
+    Field lateTriggerUpdaterField = Trigger.class.getDeclaredField("lateTriggerUpdater");
+    earlyTriggerField.setAccessible(true);
+    lateTriggerField.setAccessible(true);
+    timerTriggerField.setAccessible(true);
+    earlyTriggerUpdaterField.setAccessible(true);
+    lateTriggerUpdaterField.setAccessible(true);
+
+    assertEquals(earlyTrigger, earlyTriggerField.get(trigger));
+    assertEquals(timerTrigger, timerTriggerField.get(trigger));
+    assertEquals(lateTrigger, lateTriggerField.get(trigger));
+    assertEquals(earlyTriggerUpdater, earlyTriggerUpdaterField.get(trigger));
+    assertEquals(lateTriggerUpdater, lateTriggerUpdaterField.get(trigger));
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/windows/TestTriggerBuilder.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/windows/TestTriggerBuilder.java b/samza-api/src/test/java/org/apache/samza/operators/windows/TestTriggerBuilder.java
new file mode 100644
index 0000000..6a9b55d
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/operators/windows/TestTriggerBuilder.java
@@ -0,0 +1,226 @@
+/*
+ * 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.samza.operators.windows;
+
+import org.apache.samza.operators.TestMessageEnvelope;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
+public class TestTriggerBuilder {
+  private Field earlyTriggerField;
+  private Field lateTriggerField;
+  private Field timerTriggerField;
+  private Field earlyTriggerUpdater;
+  private Field lateTriggerUpdater;
+
+  @Before
+  public void testPrep() throws Exception {
+    this.earlyTriggerField = TriggerBuilder.class.getDeclaredField("earlyTrigger");
+    this.lateTriggerField = TriggerBuilder.class.getDeclaredField("lateTrigger");
+    this.timerTriggerField = TriggerBuilder.class.getDeclaredField("timerTrigger");
+    this.earlyTriggerUpdater = TriggerBuilder.class.getDeclaredField("earlyTriggerUpdater");
+    this.lateTriggerUpdater = TriggerBuilder.class.getDeclaredField("lateTriggerUpdater");
+
+    this.earlyTriggerField.setAccessible(true);
+    this.lateTriggerField.setAccessible(true);
+    this.timerTriggerField.setAccessible(true);
+    this.earlyTriggerUpdater.setAccessible(true);
+    this.lateTriggerUpdater.setAccessible(true);
+  }
+
+  @Test
+  public void testStaticCreators() throws NoSuchFieldException, IllegalAccessException {
+    TriggerBuilder<TestMessageEnvelope, Collection<TestMessageEnvelope>> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
+    BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean> triggerField =
+        (BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean>) this.earlyTriggerField.get(builder);
+    WindowState<Collection<TestMessageEnvelope>> mockState = mock(WindowState.class);
+    when(mockState.getNumberMessages()).thenReturn(200L);
+    assertFalse(triggerField.apply(null, mockState));
+    when(mockState.getNumberMessages()).thenReturn(2000L);
+    assertTrue(triggerField.apply(null, mockState));
+
+    Function<TestMessageEnvelope, Boolean> tokenFunc = m -> true;
+    builder = TriggerBuilder.earlyTriggerOnTokenMsg(tokenFunc);
+    triggerField = (BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean>) this.earlyTriggerField.get(builder);
+    TestMessageEnvelope m = mock(TestMessageEnvelope.class);
+    assertTrue(triggerField.apply(m, mockState));
+
+    builder = TriggerBuilder.earlyTriggerOnEventTime(tm -> tm.getMessage().getEventTime(), 30000L);
+    triggerField = (BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean>) this.earlyTriggerField.get(builder);
+    when(mockState.getEarliestEventTimeNs()).thenReturn(1000000000L);
+    when(mockState.getLatestEventTimeNs()).thenReturn(20000000000L);
+    TestMessageEnvelope.MessageType mockInnerMessage;
+    mockInnerMessage = mock(TestMessageEnvelope.MessageType.class);
+    when(mockInnerMessage.getEventTime()).thenReturn(19999000000L);
+    when(m.getMessage()).thenReturn(mockInnerMessage);
+    assertFalse(triggerField.apply(m, mockState));
+    mockInnerMessage = mock(TestMessageEnvelope.MessageType.class);
+    when(mockInnerMessage.getEventTime()).thenReturn(32000000000L);
+    when(m.getMessage()).thenReturn(mockInnerMessage);
+    assertTrue(triggerField.apply(m, mockState));
+    mockInnerMessage = mock(TestMessageEnvelope.MessageType.class);
+    when(m.getMessage()).thenReturn(mockInnerMessage);
+    when(mockInnerMessage.getEventTime()).thenReturn(1001000000L);
+    when(mockState.getLatestEventTimeNs()).thenReturn(32000000000L);
+    assertTrue(triggerField.apply(m, mockState));
+
+    BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean> mockFunc = mock(BiFunction.class);
+    builder = TriggerBuilder.earlyTrigger(mockFunc);
+    triggerField = (BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean>) this.earlyTriggerField.get(builder);
+    assertEquals(triggerField, mockFunc);
+
+    builder = TriggerBuilder.timeoutSinceFirstMessage(10000L);
+    Function<WindowState<Collection<TestMessageEnvelope>>, Boolean> timerTrigger =
+        (Function<WindowState<Collection<TestMessageEnvelope>>, Boolean>) this.timerTriggerField.get(builder);
+    when(mockState.getFirstMessageTimeNs()).thenReturn(0L);
+    assertTrue(timerTrigger.apply(mockState));
+    // set the firstMessageTimeNs to 9 second earlier, giving the test 1 second to fire up the timerTrigger before assertion
+    when(mockState.getFirstMessageTimeNs()).thenReturn(TimeUnit.MILLISECONDS.toNanos(System.currentTimeMillis() - 9000L));
+    assertFalse(timerTrigger.apply(mockState));
+
+    builder = TriggerBuilder.timeoutSinceLastMessage(10000L);
+    timerTrigger = (Function<WindowState<Collection<TestMessageEnvelope>>, Boolean>) this.timerTriggerField.get(builder);
+    when(mockState.getLastMessageTimeNs()).thenReturn(0L);
+    assertTrue(timerTrigger.apply(mockState));
+    // set the lastMessageTimeNs to 9 second earlier, giving the test 1 second to fire up the timerTrigger before assertion
+    when(mockState.getLastMessageTimeNs()).thenReturn(TimeUnit.MILLISECONDS.toNanos(System.currentTimeMillis() - 9000));
+    assertFalse(timerTrigger.apply(mockState));
+  }
+
+  @Test
+  public void testAddTimerTriggers() throws IllegalAccessException {
+    TriggerBuilder<TestMessageEnvelope, Collection<TestMessageEnvelope>> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
+    builder.addTimeoutSinceFirstMessage(10000L);
+    // exam that both earlyTrigger and timer triggers are set up
+    BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean> triggerField =
+        (BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean>) this.earlyTriggerField.get(builder);
+    WindowState<Collection<TestMessageEnvelope>> mockState = mock(WindowState.class);
+    when(mockState.getNumberMessages()).thenReturn(200L);
+    assertFalse(triggerField.apply(null, mockState));
+    // check the timer trigger
+    Function<WindowState<Collection<TestMessageEnvelope>>, Boolean> timerTrigger =
+        (Function<WindowState<Collection<TestMessageEnvelope>>, Boolean>) this.timerTriggerField.get(builder);
+    when(mockState.getFirstMessageTimeNs()).thenReturn(0L);
+    assertTrue(timerTrigger.apply(mockState));
+    // set the firstMessageTimeNs to 9 second earlier, giving the test 1 second to fire up the timerTrigger before assertion
+    when(mockState.getFirstMessageTimeNs()).thenReturn(TimeUnit.MILLISECONDS.toNanos(System.currentTimeMillis() - 9000L));
+    assertFalse(timerTrigger.apply(mockState));
+
+    // exam that both early trigger and timer triggers are set up
+    builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
+    triggerField = (BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean>) this.earlyTriggerField.get(builder);
+    mockState = mock(WindowState.class);
+    when(mockState.getNumberMessages()).thenReturn(200L);
+    assertFalse(triggerField.apply(null, mockState));
+    builder.addTimeoutSinceLastMessage(20000L);
+    // check the timer trigger
+    timerTrigger = (Function<WindowState<Collection<TestMessageEnvelope>>, Boolean>) this.timerTriggerField.get(builder);
+    when(mockState.getLastMessageTimeNs()).thenReturn(0L);
+    assertTrue(timerTrigger.apply(mockState));
+    // set the firstMessageTimeNs to 9 second earlier, giving the test 1 second to fire up the timerTrigger before assertion
+    when(mockState.getLastMessageTimeNs()).thenReturn(TimeUnit.MILLISECONDS.toNanos(System.currentTimeMillis() - 9000L));
+    assertFalse(timerTrigger.apply(mockState));
+  }
+
+  @Test
+  public void testAddLateTriggers() throws IllegalAccessException {
+    TriggerBuilder<TestMessageEnvelope, Collection<TestMessageEnvelope>> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
+    builder.addLateTriggerOnSizeLimit(10000L);
+    // exam that both earlyTrigger and lateTriggers are set up
+    BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean> earlyTrigger =
+        (BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean>) this.earlyTriggerField.get(builder);
+    WindowState<Collection<TestMessageEnvelope>> mockState = mock(WindowState.class);
+    when(mockState.getNumberMessages()).thenReturn(200L);
+    assertFalse(earlyTrigger.apply(null, mockState));
+    // check the late trigger
+    BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean> lateTrigger =
+        (BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean>) this.lateTriggerField.get(builder);
+    assertFalse(lateTrigger.apply(null, mockState));
+    // set the number of messages to 10001 to trigger the late trigger
+    when(mockState.getNumberMessages()).thenReturn(10001L);
+    assertTrue(lateTrigger.apply(null, mockState));
+
+    builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
+    builder.addLateTrigger((m, s) -> s.getOutputValue().size() > 0);
+    // exam that both earlyTrigger and lateTriggers are set up
+    earlyTrigger = (BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean>) this.earlyTriggerField.get(builder);
+    mockState = mock(WindowState.class);
+    when(mockState.getNumberMessages()).thenReturn(200L);
+    assertFalse(earlyTrigger.apply(null, mockState));
+    // exam the lateTrigger
+    when(mockState.getOutputValue()).thenReturn(new ArrayList<>());
+    lateTrigger = (BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean>) this.lateTriggerField.get(builder);
+    assertFalse(lateTrigger.apply(null, mockState));
+    List<TestMessageEnvelope> mockList = mock(ArrayList.class);
+    when(mockList.size()).thenReturn(200);
+    when(mockState.getOutputValue()).thenReturn(mockList);
+    assertTrue(lateTrigger.apply(null, mockState));
+  }
+
+  @Test
+  public void testAddTriggerUpdater() throws IllegalAccessException {
+    TriggerBuilder<TestMessageEnvelope, Collection<TestMessageEnvelope>> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
+    builder.onEarlyTrigger(c -> {
+        c.clear();
+        return c;
+      });
+    List<TestMessageEnvelope> collection = new ArrayList<TestMessageEnvelope>() { {
+        for (int i = 0; i < 10; i++) {
+          this.add(new TestMessageEnvelope(String.format("key-%d", i), "string-value", System.nanoTime()));
+        }
+      } };
+    // exam that earlyTriggerUpdater is set up
+    Function<WindowState<Collection<TestMessageEnvelope>>, WindowState<Collection<TestMessageEnvelope>>> earlyTriggerUpdater =
+        (Function<WindowState<Collection<TestMessageEnvelope>>, WindowState<Collection<TestMessageEnvelope>>>) this.earlyTriggerUpdater.get(builder);
+    WindowState<Collection<TestMessageEnvelope>> mockState = mock(WindowState.class);
+    when(mockState.getOutputValue()).thenReturn(collection);
+    earlyTriggerUpdater.apply(mockState);
+    assertTrue(collection.isEmpty());
+
+    collection.add(new TestMessageEnvelope("key-to-stay", "string-to-stay", System.nanoTime()));
+    collection.add(new TestMessageEnvelope("key-to-remove", "string-to-remove", System.nanoTime()));
+    builder.onLateTrigger(c -> {
+        c.removeIf(t -> t.getKey().equals("key-to-remove"));
+        return c;
+      });
+    // check the late trigger updater
+    Function<WindowState<Collection<TestMessageEnvelope>>, WindowState<Collection<TestMessageEnvelope>>> lateTriggerUpdater =
+        (Function<WindowState<Collection<TestMessageEnvelope>>, WindowState<Collection<TestMessageEnvelope>>>) this.lateTriggerUpdater.get(builder);
+    when(mockState.getOutputValue()).thenReturn(collection);
+    lateTriggerUpdater.apply(mockState);
+    assertTrue(collection.size() == 1);
+    assertFalse(collection.get(0).isDelete());
+    assertEquals(collection.get(0).getKey(), "key-to-stay");
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowOutput.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowOutput.java b/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowOutput.java
new file mode 100644
index 0000000..7f81fc9
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowOutput.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.samza.operators.windows;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+
+public class TestWindowOutput {
+
+  @Test
+  public void testConstructor() {
+    WindowOutput<String, Integer> wndOutput = WindowOutput.of("testMsg", 10);
+    assertEquals(wndOutput.getKey(), "testMsg");
+    assertEquals(wndOutput.getMessage(), Integer.valueOf(10));
+    assertFalse(wndOutput.isDelete());
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindows.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindows.java b/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindows.java
new file mode 100644
index 0000000..26af26e
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindows.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.samza.operators.windows;
+
+import org.apache.samza.operators.TestMessageEnvelope;
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
+public class TestWindows {
+
+  @Test
+  public void testSessionWindows() throws NoSuchFieldException, IllegalAccessException {
+    // test constructing the default session window
+    Window<TestMessageEnvelope, String, Collection<TestMessageEnvelope>, WindowOutput<String, Collection<TestMessageEnvelope>>> testWnd = Windows
+        .intoSessions(
+        TestMessageEnvelope::getKey);
+    assertTrue(testWnd instanceof SessionWindow);
+    Field wndKeyFuncField = SessionWindow.class.getDeclaredField("wndKeyFunction");
+    Field aggregatorField = SessionWindow.class.getDeclaredField("aggregator");
+    wndKeyFuncField.setAccessible(true);
+    aggregatorField.setAccessible(true);
+    Function<TestMessageEnvelope, String> wndKeyFunc = (Function<TestMessageEnvelope, String>) wndKeyFuncField.get(testWnd);
+    assertEquals(wndKeyFunc.apply(new TestMessageEnvelope("test-key", "test-value", 0)), "test-key");
+    BiFunction<TestMessageEnvelope, Collection<TestMessageEnvelope>, Collection<TestMessageEnvelope>> aggrFunc =
+        (BiFunction<TestMessageEnvelope, Collection<TestMessageEnvelope>, Collection<TestMessageEnvelope>>) aggregatorField.get(testWnd);
+    TestMessageEnvelope mockMsg = mock(TestMessageEnvelope.class);
+    Collection<TestMessageEnvelope> collection = aggrFunc.apply(mockMsg, new ArrayList<>());
+    assertTrue(collection.size() == 1);
+    assertTrue(collection.contains(mockMsg));
+
+    // test constructing the session window w/ customized session info
+    Window<TestMessageEnvelope, String, Collection<Character>, WindowOutput<String, Collection<Character>>> testWnd2 = Windows.intoSessions(
+        m -> String.format("key-%d", m.getMessage().getEventTime()), m -> m.getMessage().getValue().charAt(0));
+    assertTrue(testWnd2 instanceof SessionWindow);
+    wndKeyFunc = (Function<TestMessageEnvelope, String>) wndKeyFuncField.get(testWnd2);
+    aggrFunc = (BiFunction<TestMessageEnvelope, Collection<TestMessageEnvelope>, Collection<TestMessageEnvelope>>) aggregatorField.get(testWnd2);
+    assertEquals(wndKeyFunc.apply(new TestMessageEnvelope("test-key", "test-value", 0)), "key-0");
+    TestMessageEnvelope.MessageType mockInnerMessage = mock(TestMessageEnvelope.MessageType.class);
+    when(mockMsg.getMessage()).thenReturn(mockInnerMessage);
+    when(mockInnerMessage.getValue()).thenReturn("x-001");
+    collection = aggrFunc.apply(mockMsg, new ArrayList<>());
+    assertTrue(collection.size() == 1);
+    assertTrue(collection.contains('x'));
+
+    // test constructing session window w/ a default counter
+    Window<TestMessageEnvelope, String, Integer, WindowOutput<String, Integer>> testCounter = Windows.intoSessionCounter(
+        m -> String.format("key-%d", m.getMessage().getEventTime()));
+    assertTrue(testCounter instanceof SessionWindow);
+    wndKeyFunc = (Function<TestMessageEnvelope, String>) wndKeyFuncField.get(testCounter);
+    BiFunction<TestMessageEnvelope, Integer, Integer> counterFn = (BiFunction<TestMessageEnvelope, Integer, Integer>) aggregatorField.get(testCounter);
+    when(mockMsg.getMessage().getEventTime()).thenReturn(12345L);
+    assertEquals(wndKeyFunc.apply(mockMsg), "key-12345");
+    assertEquals(counterFn.apply(mockMsg, 1), Integer.valueOf(2));
+  }
+
+  @Test
+  public void testSetTriggers() throws NoSuchFieldException, IllegalAccessException {
+    Window<TestMessageEnvelope, String, Integer, WindowOutput<String, Integer>> testCounter = Windows.intoSessionCounter(
+        m -> String.format("key-%d", m.getMessage().getEventTime()));
+    // test session window w/ a trigger
+    TriggerBuilder<TestMessageEnvelope, Integer> triggerBuilder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000L);
+    testCounter.setTriggers(triggerBuilder);
+    Trigger<TestMessageEnvelope, WindowState<Integer>> expectedTrigger = triggerBuilder.build();
+    Trigger<TestMessageEnvelope, WindowState<Integer>> actualTrigger = Windows.getInternalWindowFn(testCounter).getTrigger();
+    // examine all trigger fields are expected
+    Field earlyTriggerField = Trigger.class.getDeclaredField("earlyTrigger");
+    Field lateTriggerField = Trigger.class.getDeclaredField("lateTrigger");
+    Field timerTriggerField = Trigger.class.getDeclaredField("timerTrigger");
+    Field earlyTriggerUpdater = Trigger.class.getDeclaredField("earlyTriggerUpdater");
+    Field lateTriggerUpdater = Trigger.class.getDeclaredField("lateTriggerUpdater");
+    earlyTriggerField.setAccessible(true);
+    lateTriggerField.setAccessible(true);
+    timerTriggerField.setAccessible(true);
+    earlyTriggerUpdater.setAccessible(true);
+    lateTriggerUpdater.setAccessible(true);
+    assertEquals(earlyTriggerField.get(expectedTrigger), earlyTriggerField.get(actualTrigger));
+    assertEquals(lateTriggerField.get(expectedTrigger), lateTriggerField.get(actualTrigger));
+    assertEquals(timerTriggerField.get(expectedTrigger), timerTriggerField.get(actualTrigger));
+    assertEquals(earlyTriggerUpdater.get(expectedTrigger), earlyTriggerUpdater.get(actualTrigger));
+    assertEquals(lateTriggerUpdater.get(expectedTrigger), lateTriggerUpdater.get(actualTrigger));
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
new file mode 100644
index 0000000..231d3f5
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
@@ -0,0 +1,134 @@
+/*
+ * 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.samza.operators;
+
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.functions.FilterFunction;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.functions.JoinFunction;
+import org.apache.samza.operators.functions.MapFunction;
+import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.operators.spec.OperatorSpec;
+import org.apache.samza.operators.spec.OperatorSpecs;
+import org.apache.samza.operators.windows.Window;
+import org.apache.samza.operators.windows.WindowFn;
+import org.apache.samza.operators.windows.WindowOutput;
+import org.apache.samza.operators.windows.WindowState;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+
+/**
+ * The implementation for input/output {@link MessageStream}s to/from the operators.
+ * Users use the {@link MessageStream} API methods to describe and chain the operators specs.
+ *
+ * @param <M>  type of {@link MessageEnvelope}s in this {@link MessageStream}
+ */
+public class MessageStreamImpl<M extends MessageEnvelope> implements MessageStream<M> {
+
+  /**
+   * The set of operators that consume the {@link MessageEnvelope}s in this {@link MessageStream}
+   */
+  private final Set<OperatorSpec> registeredOperatorSpecs = new HashSet<>();
+
+  @Override
+  public <OM extends MessageEnvelope> MessageStream<OM> map(MapFunction<M, OM> mapFn) {
+    OperatorSpec<OM> op = OperatorSpecs.<M, OM>createStreamOperator(m -> new ArrayList<OM>() { {
+        OM r = mapFn.apply(m);
+        if (r != null) {
+          this.add(r);
+        }
+      } });
+    this.registeredOperatorSpecs.add(op);
+    return op.getOutputStream();
+  }
+
+  @Override
+  public <OM extends MessageEnvelope> MessageStream<OM> flatMap(FlatMapFunction<M, OM> flatMapFn) {
+    OperatorSpec<OM> op = OperatorSpecs.createStreamOperator(flatMapFn);
+    this.registeredOperatorSpecs.add(op);
+    return op.getOutputStream();
+  }
+
+  @Override
+  public MessageStream<M> filter(FilterFunction<M> filterFn) {
+    OperatorSpec<M> op = OperatorSpecs.<M, M>createStreamOperator(t -> new ArrayList<M>() { {
+        if (filterFn.apply(t)) {
+          this.add(t);
+        }
+      } });
+    this.registeredOperatorSpecs.add(op);
+    return op.getOutputStream();
+  }
+
+  @Override
+  public void sink(SinkFunction<M> sinkFn) {
+    this.registeredOperatorSpecs.add(OperatorSpecs.createSinkOperator(sinkFn));
+  }
+
+  @Override
+  public <WK, WV, WS extends WindowState<WV>, WM extends WindowOutput<WK, WV>> MessageStream<WM> window(
+      Window<M, WK, WV, WM> window) {
+    OperatorSpec<WM> wndOp = OperatorSpecs.createWindowOperator((WindowFn<M, WK, WS, WM>) window.getInternalWindowFn());
+    this.registeredOperatorSpecs.add(wndOp);
+    return wndOp.getOutputStream();
+  }
+
+  @Override
+  public <K, JM extends MessageEnvelope<K, ?>, RM extends MessageEnvelope> MessageStream<RM> join(
+      MessageStream<JM> otherStream, JoinFunction<M, JM, RM> joinFn) {
+    MessageStreamImpl<RM> outputStream = new MessageStreamImpl<>();
+
+    BiFunction<M, JM, RM> parJoin1 = joinFn::apply;
+    BiFunction<JM, M, RM> parJoin2 = (m, t1) -> joinFn.apply(t1, m);
+
+    // TODO: need to add default store functions for the two partial join functions
+
+    ((MessageStreamImpl<JM>) otherStream).registeredOperatorSpecs.add(
+        OperatorSpecs.createPartialJoinOperator(parJoin2, outputStream));
+    this.registeredOperatorSpecs.add(OperatorSpecs.createPartialJoinOperator(parJoin1, outputStream));
+    return outputStream;
+  }
+
+  @Override
+  public MessageStream<M> merge(Collection<MessageStream<M>> otherStreams) {
+    MessageStreamImpl<M> outputStream = new MessageStreamImpl<>();
+
+    otherStreams.add(this);
+    otherStreams.forEach(other ->
+        ((MessageStreamImpl<M>) other).registeredOperatorSpecs.add(OperatorSpecs.createMergeOperator(outputStream)));
+    return outputStream;
+  }
+
+  /**
+   * Gets the operator specs registered to consume the output of this {@link MessageStream}. This is an internal API and
+   * should not be exposed to users.
+   *
+   * @return  a collection containing all {@link OperatorSpec}s that are registered with this {@link MessageStream}.
+   */
+  public Collection<OperatorSpec> getRegisteredOperatorSpecs() {
+    return Collections.unmodifiableSet(this.registeredOperatorSpecs);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/StateStoreImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/StateStoreImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/StateStoreImpl.java
new file mode 100644
index 0000000..2572f14
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/StateStoreImpl.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.samza.operators;
+
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.windows.StoreFunctions;
+import org.apache.samza.storage.kv.Entry;
+import org.apache.samza.storage.kv.KeyValueStore;
+import org.apache.samza.task.TaskContext;
+
+
+/**
+ * The base class for all state stores
+ */
+public class StateStoreImpl<M extends MessageEnvelope, SK, SS> {
+  private final String storeName;
+  private final StoreFunctions<M, SK, SS> storeFunctions;
+  private KeyValueStore<SK, SS> kvStore = null;
+
+  public StateStoreImpl(StoreFunctions<M, SK, SS> store, String storeName) {
+    this.storeFunctions = store;
+    this.storeName = storeName;
+  }
+
+  public void init(TaskContext context) {
+    this.kvStore = (KeyValueStore<SK, SS>) context.getStore(this.storeName);
+  }
+
+  public Entry<SK, SS> getState(M m) {
+    SK key = this.storeFunctions.getStoreKeyFn().apply(m);
+    SS state = this.kvStore.get(key);
+    return new Entry<>(key, state);
+  }
+
+  public Entry<SK, SS> updateState(M m, Entry<SK, SS> oldEntry) {
+    SS newValue = this.storeFunctions.getStateUpdaterFn().apply(m, oldEntry.getValue());
+    this.kvStore.put(oldEntry.getKey(), newValue);
+    return new Entry<>(oldEntry.getKey(), newValue);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/StreamOperatorAdaptorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/StreamOperatorAdaptorTask.java b/samza-operator/src/main/java/org/apache/samza/operators/StreamOperatorAdaptorTask.java
new file mode 100644
index 0000000..152cd92
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/StreamOperatorAdaptorTask.java
@@ -0,0 +1,105 @@
+/*
+ * 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.samza.operators;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.data.IncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.impl.OperatorImpl;
+import org.apache.samza.operators.impl.OperatorImpls;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.task.InitableTask;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.StreamTask;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+import org.apache.samza.task.WindowableTask;
+
+import java.util.HashMap;
+import java.util.Map;
+
+
+/**
+ * An {@link StreamTask} implementation that receives {@link IncomingSystemMessageEnvelope}s and propagates them
+ * through the user's stream transformations defined in {@link StreamOperatorTask#transform(Map)} using the
+ * {@link MessageStream} APIs.
+ * <p>
+ * This class brings all the operator API implementation components together and feeds the
+ * {@link IncomingSystemMessageEnvelope}s into the transformation chains.
+ * <p>
+ * It accepts an instance of the user implemented {@link StreamOperatorTask}. When its own {@link #init(Config, TaskContext)}
+ * method is called during startup, it creates a {@link MessageStreamImpl} corresponding to each of its input
+ * {@link SystemStreamPartition}s and then calls the user's {@link StreamOperatorTask#transform(Map)} method.
+ * <p>
+ * When users invoke the methods on the {@link MessageStream} API to describe their stream transformations in the
+ * {@link StreamOperatorTask#transform(Map)} method, the underlying {@link MessageStreamImpl} creates the
+ * corresponding {@link org.apache.samza.operators.spec.OperatorSpec} to record information about the desired
+ * transformation, and returns the output {@link MessageStream} to allow further transform chaining.
+ * <p>
+ * Once the user's transformation DAGs have been described for all {@link MessageStream}s (i.e., when the
+ * {@link StreamOperatorTask#transform(Map)} call returns), it calls
+ * {@link OperatorImpls#createOperatorImpls(MessageStreamImpl, TaskContext)} for each of the input
+ * {@link MessageStreamImpl}. This instantiates the {@link org.apache.samza.operators.impl.OperatorImpl} DAG
+ * corresponding to the aforementioned {@link org.apache.samza.operators.spec.OperatorSpec} DAG and returns the
+ * root node of the DAG, which this class saves.
+ * <p>
+ * Now that it has the root for the DAG corresponding to each {@link SystemStreamPartition}, it can pass the message
+ * envelopes received in {@link StreamTask#process(IncomingMessageEnvelope, MessageCollector, TaskCoordinator)} along
+ * to the appropriate root nodes. From then on, each {@link org.apache.samza.operators.impl.OperatorImpl} propagates
+ * its transformed output to the next set of {@link org.apache.samza.operators.impl.OperatorImpl}s.
+ */
+public final class StreamOperatorAdaptorTask implements StreamTask, InitableTask, WindowableTask {
+
+  /**
+   * A mapping from each {@link SystemStreamPartition} to the root node of its operator chain DAG.
+   */
+  private final Map<SystemStreamPartition, OperatorImpl<IncomingSystemMessageEnvelope, ? extends MessageEnvelope>> operatorChains = new HashMap<>();
+
+  private final StreamOperatorTask userTask;
+
+  public StreamOperatorAdaptorTask(StreamOperatorTask userTask) {
+    this.userTask = userTask;
+  }
+
+  @Override
+  public final void init(Config config, TaskContext context) throws Exception {
+    if (this.userTask instanceof InitableTask) {
+      ((InitableTask) this.userTask).init(config, context);
+    }
+    Map<SystemStreamPartition, MessageStream<IncomingSystemMessageEnvelope>> messageStreams = new HashMap<>();
+    context.getSystemStreamPartitions().forEach(ssp -> messageStreams.put(ssp, new MessageStreamImpl<>()));
+    this.userTask.transform(messageStreams);
+    messageStreams.forEach((ssp, ms) ->
+        operatorChains.put(ssp, OperatorImpls.createOperatorImpls((MessageStreamImpl<IncomingSystemMessageEnvelope>) ms, context)));
+  }
+
+  @Override
+  public final void process(IncomingMessageEnvelope ime, MessageCollector collector, TaskCoordinator coordinator) {
+    this.operatorChains.get(ime.getSystemStreamPartition())
+        .onNext(new IncomingSystemMessageEnvelope(ime), collector, coordinator);
+  }
+
+  @Override
+  public final void window(MessageCollector collector, TaskCoordinator coordinator) throws Exception {
+    if (this.userTask instanceof WindowableTask) {
+      ((WindowableTask) this.userTask).window(collector, coordinator);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
deleted file mode 100644
index 1eee2dc..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
+++ /dev/null
@@ -1,119 +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.samza.operators.impl;
-
-import org.apache.samza.storage.kv.Entry;
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.internal.Operators.Operator;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskContext;
-import org.apache.samza.task.TaskCoordinator;
-
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Set;
-
-
-/**
- * Implementation class for a chain of operators from the single input {@code source}
- *
- * @param <M>  type of message in the input stream {@code source}
- */
-public class ChainedOperators<M extends Message> {
-
-  private final Set<OperatorImpl> subscribers = new HashSet<>();
-
-  /**
-   * Private constructor
-   *
-   * @param source  the input source {@link MessageStream}
-   * @param context  the {@link TaskContext} object that we need to instantiate the state stores
-   */
-  private ChainedOperators(MessageStream<M> source, TaskContext context) {
-    // create the pipeline/topology starting from source
-    source.getSubscribers().forEach(sub -> {
-        // pass in the context s.t. stateful stream operators can initialize their stores
-        OperatorImpl subImpl = this.createAndSubscribe(sub, source, context);
-        this.subscribers.add(subImpl);
-      });
-  }
-
-  /**
-   * Private function to recursively instantiate the implementation of operators and the chains
-   *
-   * @param operator  the operator that subscribe to {@code source}
-   * @param source  the source {@link MessageStream}
-   * @param context  the context of the task
-   * @return  the implementation object of the corresponding {@code operator}
-   */
-  private OperatorImpl<M, ? extends Message> createAndSubscribe(Operator operator, MessageStream source,
-      TaskContext context) {
-    Entry<OperatorImpl<M, ? extends Message>, Boolean> factoryEntry = OperatorFactory.getOperator(operator);
-    if (factoryEntry.getValue()) {
-      // The operator has already been instantiated and we do not need to traverse and create the subscribers any more.
-      return factoryEntry.getKey();
-    }
-    OperatorImpl<M, ? extends Message> opImpl = factoryEntry.getKey();
-    MessageStream outStream = operator.getOutputStream();
-    Collection<Operator> subs = outStream.getSubscribers();
-    subs.forEach(sub -> {
-        OperatorImpl subImpl = this.createAndSubscribe(sub, operator.getOutputStream(), context);
-        opImpl.subscribe(subImpl);
-      });
-    // initialize the operator's state store
-    opImpl.init(source, context);
-    return opImpl;
-  }
-
-  /**
-   * Static method to create a {@link ChainedOperators} from the {@code source} stream
-   *
-   * @param source  the input source {@link MessageStream}
-   * @param context  the {@link TaskContext} object used to initialize the {@link StateStoreImpl}
-   * @param <M>  the type of input {@link Message}
-   * @return a {@link ChainedOperators} object takes the {@code source} as input
-   */
-  public static <M extends Message> ChainedOperators create(MessageStream<M> source, TaskContext context) {
-    return new ChainedOperators<>(source, context);
-  }
-
-  /**
-   * Method to navigate the incoming {@code message} through the processing chains
-   *
-   * @param message  the incoming message to this {@link ChainedOperators}
-   * @param collector  the {@link MessageCollector} object within the process context
-   * @param coordinator  the {@link TaskCoordinator} object within the process context
-   */
-  public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
-    this.subscribers.forEach(sub -> sub.onNext(message, collector, coordinator));
-  }
-
-  /**
-   * Method to handle timer events
-   *
-   * @param collector  the {@link MessageCollector} object within the process context
-   * @param coordinator  the {@link TaskCoordinator} object within the process context
-   */
-  public void onTimer(MessageCollector collector, TaskCoordinator coordinator) {
-    long nanoTime = System.nanoTime();
-    this.subscribers.forEach(sub -> sub.onTimer(nanoTime, collector, coordinator));
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorFactory.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorFactory.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorFactory.java
deleted file mode 100644
index ea90878..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorFactory.java
+++ /dev/null
@@ -1,85 +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.samza.operators.impl;
-
-import org.apache.samza.storage.kv.Entry;
-import org.apache.samza.operators.WindowState;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.internal.Operators.*;
-import org.apache.samza.operators.internal.WindowOutput;
-import org.apache.samza.operators.impl.join.PartialJoinOpImpl;
-import org.apache.samza.operators.impl.window.SessionWindowImpl;
-
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-
-/**
- * The factory class that instantiates all implementation of {@link OperatorImpl} classes.
- */
-public class OperatorFactory {
-
-  /**
-   * the static operatorMap that includes all operator implementation instances
-   */
-  private static final Map<Operator, OperatorImpl<? extends Message, ? extends Message>> OPERATOR_MAP = new ConcurrentHashMap<>();
-
-  /**
-   * The method to actually create the implementation instances of operators
-   *
-   * @param operator  the immutable definition of {@link Operator}
-   * @param <M>  type of input {@link Message}
-   * @param <RM>  type of output {@link Message}
-   * @return  the implementation object of {@link OperatorImpl}
-   */
-  private static <M extends Message, RM extends Message> OperatorImpl<M, ? extends Message> createOperator(Operator<RM> operator) {
-    if (operator instanceof StreamOperator) {
-      return new SimpleOperatorImpl<>((StreamOperator<M, RM>) operator);
-    } else if (operator instanceof SinkOperator) {
-      return new SinkOperatorImpl<>((SinkOperator<M>) operator);
-    } else if (operator instanceof WindowOperator) {
-      return new SessionWindowImpl<>((WindowOperator<M, ?, ? extends WindowState, ? extends WindowOutput>) operator);
-    } else if (operator instanceof PartialJoinOperator) {
-      return new PartialJoinOpImpl<>((PartialJoinOperator) operator);
-    }
-    throw new IllegalArgumentException(
-        String.format("The type of operator is not supported. Operator class name: %s", operator.getClass().getName()));
-  }
-
-  /**
-   * The method to get the unique implementation instance of {@link Operator}
-   *
-   * @param operator  the {@link Operator} to instantiate
-   * @param <M>  type of input {@link Message}
-   * @param <RM>  type of output {@link Message}
-   * @return  A pair of entry that include the unique implementation instance to the {@code operator} and a boolean value indicating whether
-   *          the operator instance has already been created or not. True means the operator instance has already created, false means the operator
-   *          was not created.
-   */
-  public static <M extends Message, RM extends Message> Entry<OperatorImpl<M, ? extends Message>, Boolean> getOperator(Operator<RM> operator) {
-    if (!OPERATOR_MAP.containsKey(operator)) {
-      OperatorImpl<M, ? extends Message> operatorImpl = OperatorFactory.createOperator(operator);
-      if (OPERATOR_MAP.putIfAbsent(operator, operatorImpl) == null) {
-        return new Entry<OperatorImpl<M, ? extends Message>, Boolean>(operatorImpl, false) { };
-      }
-    }
-    return new Entry<OperatorImpl<M, ? extends Message>, Boolean>((OperatorImpl<M, ? extends Message>) OPERATOR_MAP.get(operator), true) { };
-  }
-
-}


[5/6] samza git commit: SAMZA-1054: Refactor Operator APIs

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/internal/Operators.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/internal/Operators.java b/samza-api/src/main/java/org/apache/samza/operators/internal/Operators.java
deleted file mode 100644
index f06387c..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/internal/Operators.java
+++ /dev/null
@@ -1,469 +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.samza.operators.internal;
-
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.WindowState;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.storage.kv.Entry;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.UUID;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-
-
-/**
- * This class defines all basic stream operator classes used by internal implementation only. All classes defined in
- * this file are immutable.
- *
- * NOTE: Programmers should not use the operators defined in this class directly. All {@link Operator} objects
- * should be initiated via {@link MessageStream} API methods
- */
-public class Operators {
-  /**
-   * Private constructor to prevent instantiation of the {@link Operators} class
-   */
-  private Operators() {}
-
-  private static String getOperatorId() {
-    // TODO: need to change the IDs to be a consistent, durable IDs that can be recovered across container and job restarts
-    return UUID.randomUUID().toString();
-  }
-
-  /**
-   * Private interface for stream operator functions. The interface class defines the output of the stream operator function.
-   *
-   */
-  public interface Operator<OM extends Message> {
-    MessageStream<OM> getOutputStream();
-  }
-
-  /**
-   * Linear stream operator function that takes 1 input {@link Message} and output a collection of output {@link Message}s.
-   *
-   * @param <M>  the type of input {@link Message}
-   * @param <OM>  the type of output {@link Message}
-   */
-  public static class StreamOperator<M extends Message, OM extends Message> implements Operator<OM> {
-    /**
-     * The output {@link MessageStream}
-     */
-    private final MessageStream<OM> outputStream;
-
-    /**
-     * The transformation function
-     */
-    private final Function<M, Collection<OM>> txfmFunction;
-
-    /**
-     * Constructor of {@link StreamOperator}. Make it private s.t. it can only be created within {@link Operators}.
-     *
-     * @param transformFn  the transformation function to be applied that transforms 1 input {@link Message} into a collection
-     *                     of output {@link Message}s
-     */
-    private StreamOperator(Function<M, Collection<OM>> transformFn) {
-      this(transformFn, new MessageStream<>());
-    }
-
-    /**
-     * Constructor of {@link StreamOperator} which allows the user to define the output {@link MessageStream}
-     *
-     * @param transformFn  the transformation function
-     * @param outputStream  the output {@link MessageStream}
-     */
-    private StreamOperator(Function<M, Collection<OM>> transformFn, MessageStream<OM> outputStream) {
-      this.outputStream = outputStream;
-      this.txfmFunction = transformFn;
-    }
-
-    @Override
-    public MessageStream<OM> getOutputStream() {
-      return this.outputStream;
-    }
-
-    /**
-     * Method to get the transformation function.
-     *
-     * @return the {@code txfmFunction}
-     */
-    public Function<M, Collection<OM>> getFunction() {
-      return this.txfmFunction;
-    }
-
-  }
-
-  /**
-   * A sink operator function that allows customized code to send the output to external system. This is the terminal
-   * operator that does not have any output {@link MessageStream} that allows further processing in the same
-   * {@link org.apache.samza.operators.task.StreamOperatorTask}
-   *
-   * @param <M>  the type of input {@link Message}
-   */
-  public static class SinkOperator<M extends Message> implements Operator {
-
-    /**
-     * The user-defined sink function
-     */
-    private final MessageStream.VoidFunction3<M, MessageCollector, TaskCoordinator> sink;
-
-    /**
-     * Default constructor for {@link SinkOperator}. Make it private s.t. it can only be created within {@link Operators}.
-     *
-     * @param sink  the user-defined sink function
-     */
-    private SinkOperator(MessageStream.VoidFunction3<M, MessageCollector, TaskCoordinator> sink) {
-      this.sink = sink;
-    }
-
-    @Override
-    public MessageStream getOutputStream() {
-      return null;
-    }
-
-    /**
-     * Method to get the user-defined function implements the {@link SinkOperator}
-     *
-     * @return a {@link MessageStream.VoidFunction3} function that allows the caller to pass in an input message, {@link MessageCollector}
-     *         and {@link TaskCoordinator} to the sink function
-     */
-    public MessageStream.VoidFunction3<M, MessageCollector, TaskCoordinator> getFunction() {
-      return this.sink;
-    }
-  }
-
-  /**
-   * The store functions that are used by {@link WindowOperator} and {@link PartialJoinOperator} to store and retrieve
-   * buffered messages and partial aggregation results
-   *
-   * @param <SK>  the type of key used to store the operator states
-   * @param <SS>  the type of operator state. e.g. could be the partial aggregation result for a window, or a buffered
-   *             input message from the join stream for a join
-   */
-  public static class StoreFunctions<M extends Message, SK, SS> {
-    /**
-     * Function to define the key to query in the operator state store, according to the incoming {@link Message}
-     * This method only supports finding the unique key for the incoming message, which supports use case of non-overlapping
-     * windows and unique-key-based join.
-     *
-     * TODO: for windows that overlaps (i.e. sliding windows and hopping windows) and non-unique-key-based join, the query
-     * to the state store is usually a range scan. We need to add a rangeKeyFinder function to map from a single input
-     * message to a range of keys in the store.
-     */
-    private final Function<M, SK> storeKeyFinder;
-
-    /**
-     * Function to update the store entry based on the current state and the incoming {@link Message}
-     *
-     * TODO: this is assuming a 1:1 mapping from the input message to the store entry. When implementing sliding/hopping
-     * windows and non-unique-key-based join, we may need to include the corresponding state key, in addition to the
-     * state value.
-     */
-    private final BiFunction<M, SS, SS> stateUpdater;
-
-    /**
-     * Constructor of state store functions.
-     *
-     */
-    private StoreFunctions(Function<M, SK> keyFinder,
-        BiFunction<M, SS, SS> stateUpdater) {
-      this.storeKeyFinder = keyFinder;
-      this.stateUpdater = stateUpdater;
-    }
-
-    /**
-     * Method to get the {@code storeKeyFinder} function
-     *
-     * @return  the function to calculate the key from an input {@link Message}
-     */
-    public Function<M, SK> getStoreKeyFinder() {
-      return this.storeKeyFinder;
-    }
-
-    /**
-     * Method to get the {@code stateUpdater} function
-     *
-     * @return  the function to update the corresponding state according to an input {@link Message}
-     */
-    public BiFunction<M, SS, SS> getStateUpdater() {
-      return this.stateUpdater;
-    }
-  }
-
-  /**
-   * Defines a window operator function that takes one {@link MessageStream} as an input, accumulate the window state, and generate
-   * an output {@link MessageStream} w/ output type {@code WM} which extends {@link WindowOutput}
-   *
-   * @param <M>  the type of input {@link Message}
-   * @param <WK>  the type of key in the output {@link Message} from the {@link WindowOperator} function
-   * @param <WS>  the type of window state in the {@link WindowOperator} function
-   * @param <WM>  the type of window output {@link Message}
-   */
-  public static class WindowOperator<M extends Message, WK, WS extends WindowState, WM extends WindowOutput<WK, ?>> implements Operator<WM> {
-    /**
-     * The output {@link MessageStream}
-     */
-    private final MessageStream<WM> outputStream;
-
-    /**
-     * The main window transformation function that takes {@link Message}s from one input stream, aggregates w/ the window
-     * state(s) from the window state store, and generate output {@link Message}s to the output stream.
-     */
-    private final BiFunction<M, Entry<WK, WS>, WM> txfmFunction;
-
-    /**
-     * The state store functions for the {@link WindowOperator}
-     */
-    private final StoreFunctions<M, WK, WS> storeFunctions;
-
-    /**
-     * The window trigger function
-     */
-    private final Trigger<M, WS> trigger;
-
-    /**
-     * The unique ID of stateful operators
-     */
-    private final String opId;
-
-    /**
-     * Constructor for {@link WindowOperator}. Make it private s.t. it can only be created within {@link Operators}.
-     *
-     * @param windowFn  description of the window function
-     * @param operatorId  auto-generated unique ID of the operator
-     */
-    private WindowOperator(WindowFn<M, WK, WS, WM> windowFn, String operatorId) {
-      this.outputStream = new MessageStream<>();
-      this.txfmFunction = windowFn.getTransformFunc();
-      this.storeFunctions = windowFn.getStoreFuncs();
-      this.trigger = windowFn.getTrigger();
-      this.opId = operatorId;
-    }
-
-    @Override
-    public String toString() {
-      return this.opId;
-    }
-
-    @Override
-    public MessageStream<WM> getOutputStream() {
-      return this.outputStream;
-    }
-
-    /**
-     * Method to get the window's {@link StoreFunctions}.
-     *
-     * @return  the window operator's {@code storeFunctions}
-     */
-    public StoreFunctions<M, WK, WS> getStoreFunctions() {
-      return this.storeFunctions;
-    }
-
-    /**
-     * Method to get the window operator's main function
-     *
-     * @return   the window operator's {@code txfmFunction}
-     */
-    public BiFunction<M, Entry<WK, WS>, WM> getFunction() {
-      return this.txfmFunction;
-    }
-
-    /**
-     * Method to get the trigger functions
-     *
-     * @return  the {@link Trigger} for this {@link WindowOperator}
-     */
-    public Trigger<M, WS> getTrigger() {
-      return this.trigger;
-    }
-
-    /**
-     * Method to generate the window operator's state store name
-     *
-     * @param inputStream the input {@link MessageStream} to this state store
-     * @return   the persistent store name of the window operator
-     */
-    public String getStoreName(MessageStream<M> inputStream) {
-      //TODO: need to get the persistent name of ds and the operator in a serialized form
-      return String.format("input-%s-wndop-%s", inputStream.toString(), this.toString());
-    }
-  }
-
-  /**
-   * The partial join operator that takes {@link Message}s from one input stream and join w/ buffered {@link Message}s from
-   * another stream and generate join output to {@code output}
-   *
-   * @param <M>  the type of input {@link Message}
-   * @param <K>  the type of join key
-   * @param <JM>  the type of message of {@link Message} in the other join stream
-   * @param <RM>  the type of message of {@link Message} in the join output stream
-   */
-  public static class PartialJoinOperator<M extends Message<K, ?>, K, JM extends Message<K, ?>, RM extends Message> implements Operator<RM> {
-
-    private final MessageStream<RM> joinOutput;
-
-    /**
-     * The main transformation function of {@link PartialJoinOperator} that takes a type {@code M} input message,
-     * join w/ a stream of buffered {@link Message}s from another stream w/ type {@code JM}, and generate joined type {@code RM}.
-     */
-    private final BiFunction<M, JM, RM> txfmFunction;
-
-    /**
-     * The message store functions that read the buffered messages from the other stream in the join
-     */
-    private final StoreFunctions<JM, K, JM> joinStoreFunctions;
-
-    /**
-     * The message store functions that save the buffered messages of this {@link MessageStream} in the join
-     */
-    private final StoreFunctions<M, K, M> selfStoreFunctions;
-
-    /**
-     * The unique ID for the stateful operator
-     */
-    private final String opId;
-
-    /**
-     * Default constructor to create a {@link PartialJoinOperator} object
-     *
-     * @param partialJoin  partial join function that take type {@code M} of input {@link Message} and join w/ type
-     *                     {@code JM} of buffered {@link Message} from another stream
-     * @param joinOutput  the output {@link MessageStream} of the join results
-     */
-    private PartialJoinOperator(BiFunction<M, JM, RM> partialJoin, MessageStream<RM> joinOutput, String opId) {
-      this.joinOutput = joinOutput;
-      this.txfmFunction = partialJoin;
-      // Read-only join store, no creator/updater functions specified
-      this.joinStoreFunctions = new StoreFunctions<>(m -> m.getKey(), null);
-      // Buffered message store for this input stream
-      this.selfStoreFunctions = new StoreFunctions<>(m -> m.getKey(), (m, s1) -> m);
-      this.opId = opId;
-    }
-
-    @Override
-    public String toString() {
-      return this.opId;
-    }
-
-    @Override
-    public MessageStream<RM> getOutputStream() {
-      return this.joinOutput;
-    }
-
-    /**
-     * Method to get {@code joinStoreFunctions}
-     *
-     * @return  {@code joinStoreFunctions}
-     */
-    public StoreFunctions<JM, K, JM> getJoinStoreFunctions() {
-      return this.joinStoreFunctions;
-    }
-
-    /**
-     * Method to get {@code selfStoreFunctions}
-     *
-     * @return  {@code selfStoreFunctions}
-     */
-    public StoreFunctions<M, K, M> getSelfStoreFunctions() {
-      return this.selfStoreFunctions;
-    }
-
-    /**
-     * Method to get {@code txfmFunction}
-     *
-     * @return  {@code txfmFunction}
-     */
-    public BiFunction<M, JM, RM> getFunction() {
-      return this.txfmFunction;
-    }
-  }
-
-  /**
-   * The method only to be used internally in {@link MessageStream} to create {@link StreamOperator}
-   *
-   * @param transformFn  the corresponding transformation function
-   * @param <M>  type of input {@link Message}
-   * @param <OM>  type of output {@link Message}
-   * @return  the {@link StreamOperator}
-   */
-  public static <M extends Message, OM extends Message> StreamOperator<M, OM> getStreamOperator(Function<M, Collection<OM>> transformFn) {
-    return new StreamOperator<>(transformFn);
-  }
-
-  /**
-   * The method only to be used internally in {@link MessageStream} to create {@link SinkOperator}
-   *
-   * @param sinkFn  the sink function
-   * @param <M>  type of input {@link Message}
-   * @return   the {@link SinkOperator}
-   */
-  public static <M extends Message> SinkOperator<M> getSinkOperator(MessageStream.VoidFunction3<M, MessageCollector, TaskCoordinator> sinkFn) {
-    return new SinkOperator<>(sinkFn);
-  }
-
-  /**
-   * The method only to be used internally in {@link MessageStream} to create {@link WindowOperator}
-   *
-   * @param windowFn  the {@link WindowFn} function
-   * @param <M>  type of input {@link Message}
-   * @param <WK>  type of window key
-   * @param <WS>  type of {@link WindowState}
-   * @param <WM>  type of output {@link WindowOutput}
-   * @return  the {@link WindowOperator}
-   */
-  public static <M extends Message, WK, WS extends WindowState, WM extends WindowOutput<WK, ?>> WindowOperator<M, WK, WS, WM> getWindowOperator(
-      WindowFn<M, WK, WS, WM> windowFn) {
-    return new WindowOperator<>(windowFn, Operators.getOperatorId());
-  }
-
-  /**
-   * The method only to be used internally in {@link MessageStream} to create {@link WindowOperator}
-   *
-   * @param joiner  the {@link WindowFn} function
-   * @param joinOutput  the output {@link MessageStream}
-   * @param <M>  type of input {@link Message}
-   * @param <K>  type of join key
-   * @param <JM>  the type of message in the {@link Message} from the other join stream
-   * @param <RM>  the type of message in the {@link Message} from the join function
-   * @return  the {@link PartialJoinOperator}
-   */
-  public static <M extends Message<K, ?>, K, JM extends Message<K, ?>, RM extends Message> PartialJoinOperator<M, K, JM, RM> getPartialJoinOperator(
-      BiFunction<M, JM, RM> joiner, MessageStream<RM> joinOutput) {
-    return new PartialJoinOperator<>(joiner, joinOutput, Operators.getOperatorId());
-  }
-
-  /**
-   * The method only to be used internally in {@link MessageStream} to create {@link StreamOperator} as a merger function
-   *
-   * @param mergeOutput  the common output {@link MessageStream} from the merger
-   * @param <M>  the type of input {@link Message}
-   * @return  the {@link StreamOperator} for merge
-   */
-  public static <M extends Message> StreamOperator<M, M> getMergeOperator(MessageStream<M> mergeOutput) {
-    return new StreamOperator<M, M>(t ->
-      new ArrayList<M>() { {
-          this.add(t);
-        } },
-      mergeOutput);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/internal/Trigger.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/internal/Trigger.java b/samza-api/src/main/java/org/apache/samza/operators/internal/Trigger.java
deleted file mode 100644
index 3b50e2b..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/internal/Trigger.java
+++ /dev/null
@@ -1,95 +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.samza.operators.internal;
-
-import org.apache.samza.operators.WindowState;
-import org.apache.samza.operators.data.Message;
-
-import java.util.function.BiFunction;
-import java.util.function.Function;
-
-/**
- * Defines the trigger functions for {@link Operators.WindowOperator}. This class is immutable.
- *
- * @param <M>  the type of message from the input stream
- * @param <S>  the type of state variable in the window's state store
- */
-public class Trigger<M extends Message, S extends WindowState> {
-
-  /**
-   * System timer based trigger condition. This is the only guarantee that the {@link Operators.WindowOperator} will proceed forward
-   */
-  private final Function<S, Boolean> timerTrigger;
-
-  /**
-   * early trigger condition that determines when to send the first output from the {@link Operators.WindowOperator}
-   */
-  private final BiFunction<M, S, Boolean> earlyTrigger;
-
-  /**
-   * late trigger condition that determines when to send the updated output after the first one from a {@link Operators.WindowOperator}
-   */
-  private final BiFunction<M, S, Boolean> lateTrigger;
-
-  /**
-   * the function to updated the window state when the first output is triggered
-   */
-  private final Function<S, S> earlyTriggerUpdater;
-
-  /**
-   * the function to updated the window state when the late output is triggered
-   */
-  private final Function<S, S> lateTriggerUpdater;
-
-  /**
-   * Private constructor to prevent instantiation
-   *
-   * @param timerTrigger  system timer trigger condition
-   * @param earlyTrigger  early trigger condition
-   * @param lateTrigger   late trigger condition
-   * @param earlyTriggerUpdater  early trigger state updater
-   * @param lateTriggerUpdater   late trigger state updater
-   */
-  private Trigger(Function<S, Boolean> timerTrigger, BiFunction<M, S, Boolean> earlyTrigger, BiFunction<M, S, Boolean> lateTrigger,
-      Function<S, S> earlyTriggerUpdater, Function<S, S> lateTriggerUpdater) {
-    this.timerTrigger = timerTrigger;
-    this.earlyTrigger = earlyTrigger;
-    this.lateTrigger = lateTrigger;
-    this.earlyTriggerUpdater = earlyTriggerUpdater;
-    this.lateTriggerUpdater = lateTriggerUpdater;
-  }
-
-  /**
-   * Static method to create a {@link Trigger} object
-   *
-   * @param timerTrigger  system timer trigger condition
-   * @param earlyTrigger  early trigger condition
-   * @param lateTrigger  late trigger condition
-   * @param earlyTriggerUpdater  early trigger state updater
-   * @param lateTriggerUpdater  late trigger state updater
-   * @param <M>  the type of input {@link Message}
-   * @param <S>  the type of window state extends {@link WindowState}
-   * @return  the {@link Trigger} function
-   */
-  public static <M extends Message, S extends WindowState> Trigger<M, S> createTrigger(Function<S, Boolean> timerTrigger,
-      BiFunction<M, S, Boolean> earlyTrigger, BiFunction<M, S, Boolean> lateTrigger, Function<S, S> earlyTriggerUpdater,
-      Function<S, S> lateTriggerUpdater) {
-    return new Trigger(timerTrigger, earlyTrigger, lateTrigger, earlyTriggerUpdater, lateTriggerUpdater);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/internal/WindowFn.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/internal/WindowFn.java b/samza-api/src/main/java/org/apache/samza/operators/internal/WindowFn.java
deleted file mode 100644
index 489e5b8..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/internal/WindowFn.java
+++ /dev/null
@@ -1,60 +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.samza.operators.internal;
-
-import org.apache.samza.operators.WindowState;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.storage.kv.Entry;
-
-import java.util.function.BiFunction;
-
-
-/**
- * Defines an internal representation of a window function. This class SHOULD NOT be used by the programmer directly. It is used
- * by the internal representation and implementation classes in operators.
- *
- * @param <M> type of input stream {@link Message} for window
- * @param <WK>  type of window key in the output {@link Message}
- * @param <WS>  type of {@link WindowState} variable in the state store
- * @param <WM>  type of the message in the output stream
- */
-public interface WindowFn<M extends Message, WK, WS extends WindowState, WM extends WindowOutput<WK, ?>> {
-
-  /**
-   * get the transformation function of the {@link WindowFn}
-   *
-   * @return  the transformation function takes type {@code M} message and the window state entry, then transform to an {@link WindowOutput}
-   */
-  BiFunction<M, Entry<WK, WS>, WM> getTransformFunc();
-
-  /**
-   * get the state store functions for this {@link WindowFn}
-   *
-   * @return  the collection of state store methods
-   */
-  Operators.StoreFunctions<M, WK, WS> getStoreFuncs();
-
-  /**
-   * get the trigger conditions for this {@link WindowFn}
-   *
-   * @return  the trigger condition for the {@link WindowFn} function
-   */
-  Trigger<M, WS> getTrigger();
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/internal/WindowOutput.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/internal/WindowOutput.java b/samza-api/src/main/java/org/apache/samza/operators/internal/WindowOutput.java
deleted file mode 100644
index 643b703..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/internal/WindowOutput.java
+++ /dev/null
@@ -1,55 +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.samza.operators.internal;
-
-import org.apache.samza.operators.data.Message;
-
-
-/**
- * This class defines the specific type of output messages from a {@link Operators.WindowOperator} function
- *
- * @param <K>  the type of key in the output window result
- * @param <M>  the type of value in the output window result
- */
-public final class WindowOutput<K, M> implements Message<K, M> {
-  private final K key;
-  private final M value;
-
-  WindowOutput(K key, M aggregated) {
-    this.key = key;
-    this.value = aggregated;
-  }
-
-  @Override public M getMessage() {
-    return this.value;
-  }
-
-  @Override public K getKey() {
-    return this.key;
-  }
-
-  @Override public long getTimestamp() {
-    return 0;
-  }
-
-  static public <K, M> WindowOutput<K, M> of(K key, M result) {
-    return new WindowOutput<>(key, result);
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/task/StreamOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/task/StreamOperatorTask.java b/samza-api/src/main/java/org/apache/samza/operators/task/StreamOperatorTask.java
deleted file mode 100644
index 42c8f74..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/task/StreamOperatorTask.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.samza.operators.task;
-
-import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.MessageStreams.SystemMessageStream;
-
-import java.util.Collection;
-
-/**
- * This interface defines the methods that user needs to implement via the operator programming APIs.
- */
-@InterfaceStability.Unstable
-public interface StreamOperatorTask {
-
-  /**
-   * Defines the method for users to initialize the operator chains consuming from all {@link SystemMessageStream}s.
-   * Users have to implement this function to define their transformation logic on each of the incoming
-   * {@link SystemMessageStream}.
-   *
-   * Note that each {@link SystemMessageStream} corresponds to an input {@link org.apache.samza.system.SystemStreamPartition}
-   *
-   * @param sources  the collection of {@link SystemMessageStream}s that takes {@link org.apache.samza.operators.data.IncomingSystemMessage}
-   *                 from a {@link org.apache.samza.system.SystemStreamPartition}
-   */
-  void initOperators(Collection<SystemMessageStream> sources);
-
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/windows/SessionWindow.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/SessionWindow.java b/samza-api/src/main/java/org/apache/samza/operators/windows/SessionWindow.java
new file mode 100644
index 0000000..287025c
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/SessionWindow.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.samza.operators.windows;
+
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.storage.kv.Entry;
+
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+/**
+ * This class defines a session window function class
+ *
+ * @param <M>  the type of input {@link MessageEnvelope}
+ * @param <WK>  the type of session key in the session window
+ * @param <WV>  the type of output value in each session window
+ */
+public class SessionWindow<M extends MessageEnvelope, WK, WV> implements Window<M, WK, WV, WindowOutput<WK, WV>> {
+
+  /**
+   * Constructor. Made private s.t. it can only be instantiated via the static API methods in {@link Windows}
+   *
+   * @param sessionKeyFunction  function to get the session key from the input {@link MessageEnvelope}
+   * @param aggregator  function to calculate the output value based on the input {@link MessageEnvelope} and current output value
+   */
+  SessionWindow(Function<M, WK> sessionKeyFunction, BiFunction<M, WV, WV> aggregator) {
+    this.wndKeyFunction = sessionKeyFunction;
+    this.aggregator = aggregator;
+  }
+
+  /**
+   * function to calculate the window key from input {@link MessageEnvelope}
+   */
+  private final Function<M, WK> wndKeyFunction;
+
+  /**
+   * function to calculate the output value from the input {@link MessageEnvelope} and the current output value
+   */
+  private final BiFunction<M, WV, WV> aggregator;
+
+  /**
+   * trigger condition that determines when to send the {@link WindowOutput}
+   */
+  private Trigger<M, WindowState<WV>> trigger = null;
+
+  //TODO: need to create a set of {@link StoreFunctions} that is default to input {@link MessageEnvelope} type for {@link Window}
+  private StoreFunctions<M, WK, WindowState<WV>> storeFunctions = null;
+
+  /**
+   * Public API methods start here
+   */
+
+  /**
+   * Public API method to define the watermark trigger for the window operator
+   *
+   * @param wndTrigger {@link Trigger} function defines the watermark trigger for this {@link SessionWindow}
+   * @return The window operator w/ the defined watermark trigger
+   */
+  @Override
+  public Window<M, WK, WV, WindowOutput<WK, WV>> setTriggers(TriggerBuilder<M, WV> wndTrigger) {
+    this.trigger = wndTrigger.build();
+    return this;
+  }
+
+  private BiFunction<M, Entry<WK, WindowState<WV>>, WindowOutput<WK, WV>> getTransformFunc() {
+    // TODO: actual implementation of the main session window logic, based on the wndKeyFunction, aggregator, and triggers;
+    return null;
+  }
+
+  public WindowFn<M, WK, WindowState<WV>, WindowOutput<WK, WV>> getInternalWindowFn() {
+    return new WindowFn<M, WK, WindowState<WV>, WindowOutput<WK, WV>>() {
+
+      @Override public BiFunction<M, Entry<WK, WindowState<WV>>, WindowOutput<WK, WV>> getTransformFn() {
+        return SessionWindow.this.getTransformFunc();
+      }
+
+      @Override public StoreFunctions<M, WK, WindowState<WV>> getStoreFns() {
+        return SessionWindow.this.storeFunctions;
+      }
+
+      @Override public Trigger<M, WindowState<WV>> getTrigger() {
+        return SessionWindow.this.trigger;
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/windows/StoreFunctions.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/StoreFunctions.java b/samza-api/src/main/java/org/apache/samza/operators/windows/StoreFunctions.java
new file mode 100644
index 0000000..0d40761
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/StoreFunctions.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.samza.operators.windows;
+
+import org.apache.samza.operators.data.MessageEnvelope;
+
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+/**
+ * The store functions that are used by window and partial join operators to store and retrieve buffered {@link MessageEnvelope}s
+ * and partial aggregation results.
+ *
+ * @param <SK>  the type of key used to store the operator state
+ * @param <SS>  the type of operator state. E.g. could be the partial aggregation result for a window, or a buffered
+ *             input {@link MessageEnvelope} from the join stream for a join
+ */
+public class StoreFunctions<M extends MessageEnvelope, SK, SS> {
+  /**
+   * Function that returns the key to query in the operator state store for a particular {@link MessageEnvelope}.
+   * This 1:1 function only returns a single key for the incoming {@link MessageEnvelope}. This is sufficient to support
+   * non-overlapping windows and unique-key based joins.
+   *
+   * TODO: for windows that overlaps (i.e. sliding windows and hopping windows) and non-unique-key-based join,
+   * the query to the state store is usually a range scan. We need to add a rangeKeyFinder function
+   * (or make this function return a collection) to map from a single input {@link MessageEnvelope} to a range of keys in the store.
+   */
+  private final Function<M, SK> storeKeyFn;
+
+  /**
+   * Function to update the store entry based on the current operator state and the incoming {@link MessageEnvelope}.
+   *
+   * TODO: this is assuming a 1:1 mapping from the input {@link MessageEnvelope} to the store entry. When implementing sliding/hopping
+   * windows and non-unique-key-based join, we may need to include the corresponding state key in addition to the
+   * state value. Alternatively this can be called once for each store key for the {@link MessageEnvelope}.
+   */
+  private final BiFunction<M, SS, SS> stateUpdaterFn;
+
+  public StoreFunctions(Function<M, SK> storeKeyFn, BiFunction<M, SS, SS> stateUpdaterFn) {
+    this.storeKeyFn = storeKeyFn;
+    this.stateUpdaterFn = stateUpdaterFn;
+  }
+
+  public Function<M, SK> getStoreKeyFn() {
+    return this.storeKeyFn;
+  }
+
+  public BiFunction<M, SS, SS> getStateUpdaterFn() {
+    return this.stateUpdaterFn;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/windows/Trigger.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/Trigger.java b/samza-api/src/main/java/org/apache/samza/operators/windows/Trigger.java
new file mode 100644
index 0000000..c8b0edb
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/Trigger.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.samza.operators.windows;
+
+import org.apache.samza.operators.data.MessageEnvelope;
+
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+/**
+ * Defines the trigger functions for the window operator. This class is immutable.
+ *
+ * @param <M>  the type of {@link MessageEnvelope} in the input stream
+ * @param <S>  the type of state variable in the window's state store
+ */
+public class Trigger<M extends MessageEnvelope, S extends WindowState> {
+
+  /**
+   * System timer based trigger condition. This is the only guarantee that the window operator will proceed forward
+   */
+  private final Function<S, Boolean> timerTrigger;
+
+  /**
+   * early trigger condition that determines when to send the first output from the window operator
+   */
+  private final BiFunction<M, S, Boolean> earlyTrigger;
+
+  /**
+   * late trigger condition that determines when to send the updated output after the first one from a window operator
+   */
+  private final BiFunction<M, S, Boolean> lateTrigger;
+
+  /**
+   * the function to updated the window state when the first output is triggered
+   */
+  private final Function<S, S> earlyTriggerUpdater;
+
+  /**
+   * the function to updated the window state when the late output is triggered
+   */
+  private final Function<S, S> lateTriggerUpdater;
+
+  /**
+   * Private constructor to prevent instantiation
+   *
+   * @param timerTrigger  system timer trigger condition
+   * @param earlyTrigger  early trigger condition
+   * @param lateTrigger   late trigger condition
+   * @param earlyTriggerUpdater  early trigger state updater
+   * @param lateTriggerUpdater   late trigger state updater
+   */
+  private Trigger(Function<S, Boolean> timerTrigger, BiFunction<M, S, Boolean> earlyTrigger, BiFunction<M, S, Boolean> lateTrigger,
+      Function<S, S> earlyTriggerUpdater, Function<S, S> lateTriggerUpdater) {
+    this.timerTrigger = timerTrigger;
+    this.earlyTrigger = earlyTrigger;
+    this.lateTrigger = lateTrigger;
+    this.earlyTriggerUpdater = earlyTriggerUpdater;
+    this.lateTriggerUpdater = lateTriggerUpdater;
+  }
+
+  /**
+   * Static method to create a {@link Trigger} object
+   *
+   * @param timerTrigger  system timer trigger condition
+   * @param earlyTrigger  early trigger condition
+   * @param lateTrigger  late trigger condition
+   * @param earlyTriggerUpdater  early trigger state updater
+   * @param lateTriggerUpdater  late trigger state updater
+   * @param <M>  the type of input {@link MessageEnvelope}
+   * @param <S>  the type of window state extends {@link WindowState}
+   * @return  the {@link Trigger} function
+   */
+  public static <M extends MessageEnvelope, S extends WindowState> Trigger<M, S> createTrigger(Function<S, Boolean> timerTrigger,
+      BiFunction<M, S, Boolean> earlyTrigger, BiFunction<M, S, Boolean> lateTrigger, Function<S, S> earlyTriggerUpdater,
+      Function<S, S> lateTriggerUpdater) {
+    return new Trigger(timerTrigger, earlyTrigger, lateTrigger, earlyTriggerUpdater, lateTriggerUpdater);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/windows/TriggerBuilder.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/TriggerBuilder.java b/samza-api/src/main/java/org/apache/samza/operators/windows/TriggerBuilder.java
new file mode 100644
index 0000000..6336a50
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/TriggerBuilder.java
@@ -0,0 +1,320 @@
+/*
+ * 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.samza.operators.windows;
+
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.operators.data.MessageEnvelope;
+
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+
+/**
+ * This class defines a builder of {@link Trigger} object for a {@link Window}. The triggers are categorized into
+ * three types:
+ *
+ * <p>
+ *   early trigger: defines the condition when the first output from the window function is sent.
+ *   late trigger: defines the condition when the updated output after the first output is sent.
+ *   timer trigger: defines a system timeout condition to trigger output if no more inputs are received to enable early/late triggers
+ * </p>
+ *
+ * If multiple conditions are defined for a specific type of trigger, the aggregated trigger is the disjunction
+ * of each individual trigger (i.e. OR).
+ *
+ * @param <M>  the type of input {@link MessageEnvelope} to the {@link Window}
+ * @param <V>  the type of output value from the {@link Window}
+ */
+@InterfaceStability.Unstable
+public final class TriggerBuilder<M extends MessageEnvelope, V> {
+
+  /**
+   * Predicate helper to OR multiple trigger conditions
+   */
+  static class PredicateHelper {
+    static <M, S> BiFunction<M, S, Boolean> or(BiFunction<M, S, Boolean> lhs, BiFunction<M, S, Boolean> rhs) {
+      return (m, s) -> lhs.apply(m, s) || rhs.apply(m, s);
+    }
+
+    static <S> Function<S, Boolean> or(Function<S, Boolean> lhs, Function<S, Boolean> rhs) {
+      return s -> lhs.apply(s) || rhs.apply(s);
+    }
+  }
+
+  /**
+   * The early trigger condition that determines the first output from the {@link Window}
+   */
+  private BiFunction<M, WindowState<V>, Boolean> earlyTrigger = null;
+
+  /**
+   * The late trigger condition that determines the late output(s) from the {@link Window}
+   */
+  private BiFunction<M, WindowState<V>, Boolean> lateTrigger = null;
+
+  /**
+   * The system timer based trigger conditions that guarantees the {@link Window} proceeds forward
+   */
+  private Function<WindowState<V>, Boolean> timerTrigger = null;
+
+  /**
+   * The state updater function to be applied after the first output is triggered
+   */
+  private Function<WindowState<V>, WindowState<V>> earlyTriggerUpdater = Function.identity();
+
+  /**
+   * The state updater function to be applied after the late output is triggered
+   */
+  private Function<WindowState<V>, WindowState<V>> lateTriggerUpdater = Function.identity();
+
+  /**
+   * Helper method to add a trigger condition
+   *
+   * @param currentTrigger  current trigger condition
+   * @param newTrigger  new trigger condition
+   * @return  combined trigger condition that is {@code currentTrigger} OR {@code newTrigger}
+   */
+  private BiFunction<M, WindowState<V>, Boolean> addTrigger(BiFunction<M, WindowState<V>, Boolean> currentTrigger,
+      BiFunction<M, WindowState<V>, Boolean> newTrigger) {
+    if (currentTrigger == null) {
+      return newTrigger;
+    }
+
+    return PredicateHelper.or(currentTrigger, newTrigger);
+  }
+
+  /**
+   * Helper method to add a system timer trigger
+   *
+   * @param currentTimer  current timer condition
+   * @param newTimer  new timer condition
+   * @return  combined timer condition that is {@code currentTimer} OR {@code newTimer}
+   */
+  private Function<WindowState<V>, Boolean> addTimerTrigger(Function<WindowState<V>, Boolean> currentTimer,
+      Function<WindowState<V>, Boolean> newTimer) {
+    if (currentTimer == null) {
+      return newTimer;
+    }
+
+    return PredicateHelper.or(currentTimer, newTimer);
+  }
+
+  /**
+   * default constructor to prevent instantiation
+   */
+  private TriggerBuilder() {}
+
+  /**
+   * Constructor that set the size limit as the early trigger for a window
+   *
+   * @param sizeLimit  the number of {@link MessageEnvelope}s in a window that would trigger the first output
+   */
+  private TriggerBuilder(long sizeLimit) {
+    this.earlyTrigger = (m, s) -> s.getNumberMessages() > sizeLimit;
+  }
+
+  /**
+   * Constructor that set the event time length as the early trigger
+   *
+   * @param eventTimeFunction  the function that calculate the event time in nano-second from the input {@link MessageEnvelope}
+   * @param wndLenMs  the window length in event time in milli-second
+   */
+  private TriggerBuilder(Function<M, Long> eventTimeFunction, long wndLenMs) {
+    this.earlyTrigger = (m, s) ->
+        TimeUnit.NANOSECONDS.toMillis(Math.max(s.getLatestEventTimeNs() - s.getEarliestEventTimeNs(),
+            eventTimeFunction.apply(m) - s.getEarliestEventTimeNs())) > wndLenMs;
+  }
+
+  /**
+   * Constructor that set the special token {@link MessageEnvelope} as the early trigger
+   *
+   * @param tokenFunc  the function that checks whether an input {@link MessageEnvelope} is a token {@link MessageEnvelope} that triggers window output
+   */
+  private TriggerBuilder(Function<M, Boolean> tokenFunc) {
+    this.earlyTrigger = (m, s) -> tokenFunc.apply(m);
+  }
+
+  /**
+   * Build method that creates an {@link Trigger} object based on the trigger conditions set in {@link TriggerBuilder}
+   * This is kept package private and only used by {@link Windows} to convert the mutable {@link TriggerBuilder} object to an immutable {@link Trigger} object
+   *
+   * @return  the final {@link Trigger} object
+   */
+  Trigger<M, WindowState<V>> build() {
+    return Trigger.createTrigger(this.timerTrigger, this.earlyTrigger, this.lateTrigger, this.earlyTriggerUpdater, this.lateTriggerUpdater);
+  }
+
+  /**
+   * Public API methods start here
+   */
+
+
+  /**
+   * API method to allow users to set an update method to update the output value after the first window output is triggered
+   * by the early trigger condition
+   *
+   * @param onTriggerFunc  the method to update the output value after the early trigger
+   * @return  the {@link TriggerBuilder} object
+   */
+  public TriggerBuilder<M, V> onEarlyTrigger(Function<V, V> onTriggerFunc) {
+    this.earlyTriggerUpdater = s -> {
+      s.setOutputValue(onTriggerFunc.apply(s.getOutputValue()));
+      return s;
+    };
+    return this;
+  }
+
+  /**
+   * API method to allow users to set an update method to update the output value after a late window output is triggered
+   * by the late trigger condition
+   *
+   * @param onTriggerFunc  the method to update the output value after the late trigger
+   * @return  the {@link TriggerBuilder} object
+   */
+  public TriggerBuilder<M, V> onLateTrigger(Function<V, V> onTriggerFunc) {
+    this.lateTriggerUpdater = s -> {
+      s.setOutputValue(onTriggerFunc.apply(s.getOutputValue()));
+      return s;
+    };
+    return this;
+  }
+
+  /**
+   * API method to allow users to add a system timer trigger based on timeout after the last {@link MessageEnvelope} received in the window
+   *
+   * @param timeoutMs  the timeout in ms after the last {@link MessageEnvelope} received in the window
+   * @return  the {@link TriggerBuilder} object
+   */
+  public TriggerBuilder<M, V> addTimeoutSinceLastMessage(long timeoutMs) {
+    this.timerTrigger = this.addTimerTrigger(this.timerTrigger,
+        s -> TimeUnit.NANOSECONDS.toMillis(s.getLastMessageTimeNs()) + timeoutMs < System.currentTimeMillis());
+    return this;
+  }
+
+  /**
+   * API method to allow users to add a system timer trigger based on the timeout after the first {@link MessageEnvelope} received in the window
+   *
+   * @param timeoutMs  the timeout in ms after the first {@link MessageEnvelope} received in the window
+   * @return  the {@link TriggerBuilder} object
+   */
+  public TriggerBuilder<M, V> addTimeoutSinceFirstMessage(long timeoutMs) {
+    this.timerTrigger = this.addTimerTrigger(this.timerTrigger, s ->
+        TimeUnit.NANOSECONDS.toMillis(s.getFirstMessageTimeNs()) + timeoutMs < System.currentTimeMillis());
+    return this;
+  }
+
+  /**
+   * API method allow users to add a late trigger based on the window size limit
+   *
+   * @param sizeLimit  limit on the number of {@link MessageEnvelope}s in window
+   * @return  the {@link TriggerBuilder} object
+   */
+  public TriggerBuilder<M, V> addLateTriggerOnSizeLimit(long sizeLimit) {
+    this.lateTrigger = this.addTrigger(this.lateTrigger, (m, s) -> s.getNumberMessages() > sizeLimit);
+    return this;
+  }
+
+  /**
+   * API method to allow users to define a customized late trigger function based on input {@link MessageEnvelope} and the window state
+   *
+   * @param lateTrigger  the late trigger condition based on input {@link MessageEnvelope} and the current {@link WindowState}
+   * @return  the {@link TriggerBuilder} object
+   */
+  public TriggerBuilder<M, V> addLateTrigger(BiFunction<M, WindowState<V>, Boolean> lateTrigger) {
+    this.lateTrigger = this.addTrigger(this.lateTrigger, lateTrigger);
+    return this;
+  }
+
+  /**
+   * Static API method to create a {@link TriggerBuilder} w/ early trigger condition based on window size limit
+   *
+   * @param sizeLimit  window size limit
+   * @param <M>  the type of input {@link MessageEnvelope}
+   * @param <V>  the type of {@link Window} output value
+   * @return  the {@link TriggerBuilder} object
+   */
+  public static <M extends MessageEnvelope, V> TriggerBuilder<M, V> earlyTriggerWhenExceedWndLen(long sizeLimit) {
+    return new TriggerBuilder<M, V>(sizeLimit);
+  }
+
+  /**
+   * Static API method to create a {@link TriggerBuilder} w/ early trigger condition based on event time window
+   *
+   *
+   * @param eventTimeFunc  the function to get the event time from the input {@link MessageEnvelope}
+   * @param eventTimeWndSizeMs  the event time window size in Ms
+   * @param <M>  the type of input {@link MessageEnvelope}
+   * @param <V>  the type of {@link Window} output value
+   * @return  the {@link TriggerBuilder} object
+   */
+  public static <M extends MessageEnvelope, V> TriggerBuilder<M, V> earlyTriggerOnEventTime(Function<M, Long> eventTimeFunc, long eventTimeWndSizeMs) {
+    return new TriggerBuilder<M, V>(eventTimeFunc, eventTimeWndSizeMs);
+  }
+
+  /**
+   * Static API method to create a {@link TriggerBuilder} w/ early trigger condition based on token {@link MessageEnvelope}s
+   *
+   * @param tokenFunc  the function to determine whether an input {@link MessageEnvelope} is a window token or not
+   * @param <M>  the type of input {@link MessageEnvelope}
+   * @param <V>  the type of {@link Window} output value
+   * @return  the {@link TriggerBuilder} object
+   */
+  public static <M extends MessageEnvelope, V> TriggerBuilder<M, V> earlyTriggerOnTokenMsg(Function<M, Boolean> tokenFunc) {
+    return new TriggerBuilder<M, V>(tokenFunc);
+  }
+
+  /**
+   * Static API method to allow customized early trigger condition based on input {@link MessageEnvelope} and the corresponding {@link WindowState}
+   *
+   * @param earlyTrigger  the user defined early trigger condition
+   * @param <M>   the input {@link MessageEnvelope} type
+   * @param <V>   the output value from the window
+   * @return   the {@link TriggerBuilder} object
+   */
+  public static <M extends MessageEnvelope, V> TriggerBuilder<M, V> earlyTrigger(BiFunction<M, WindowState<V>, Boolean> earlyTrigger) {
+    TriggerBuilder<M, V> newTriggers =  new TriggerBuilder<M, V>();
+    newTriggers.earlyTrigger = newTriggers.addTrigger(newTriggers.earlyTrigger, earlyTrigger);
+    return newTriggers;
+  }
+
+  /**
+   * Static API method to create a {@link TriggerBuilder} w/ system timeout after the last {@link MessageEnvelope} received in the window
+   *
+   * @param timeoutMs  timeout in ms after the last {@link MessageEnvelope} received
+   * @param <M>  the type of input {@link MessageEnvelope}
+   * @param <V>  the type of {@link Window} output value
+   * @return  the {@link TriggerBuilder} object
+   */
+  public static <M extends MessageEnvelope, V> TriggerBuilder<M, V> timeoutSinceLastMessage(long timeoutMs) {
+    return new TriggerBuilder<M, V>().addTimeoutSinceLastMessage(timeoutMs);
+  }
+
+  /**
+   * Static API method to create a {@link TriggerBuilder} w/ system timeout after the first {@link MessageEnvelope} received in the window
+   *
+   * @param timeoutMs  timeout in ms after the first {@link MessageEnvelope} received
+   * @param <M>  the type of input {@link MessageEnvelope}
+   * @param <V>  the type of {@link Window} output value
+   * @return  the {@link TriggerBuilder} object
+   */
+  public static <M extends MessageEnvelope, V> TriggerBuilder<M, V> timeoutSinceFirstMessage(long timeoutMs) {
+    return new TriggerBuilder<M, V>().addTimeoutSinceFirstMessage(timeoutMs);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/windows/Window.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/Window.java b/samza-api/src/main/java/org/apache/samza/operators/windows/Window.java
new file mode 100644
index 0000000..56a307d
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/Window.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.samza.operators.windows;
+
+import org.apache.samza.operators.data.MessageEnvelope;
+
+/**
+ * The public programming interface class for window function
+ *
+ * @param <M>  the type of input {@link MessageEnvelope}
+ * @param <WK>  the type of key to the {@link Window}
+ * @param <WV>  the type of output value in the {@link WindowOutput}
+ * @param <WM>  the type of {@link MessageEnvelope} in the window output stream
+ */
+public interface Window<M extends MessageEnvelope, WK, WV, WM extends WindowOutput<WK, WV>> {
+
+  /**
+   * Set the triggers for this {@link Window}
+   *
+   * @param wndTrigger  trigger conditions set by the programmers
+   * @return  the {@link Window} function w/ the trigger {@code wndTrigger}
+   */
+  Window<M, WK, WV, WM> setTriggers(TriggerBuilder<M, WV> wndTrigger);
+
+  /**
+   * Internal implementation helper to get the functions associated with this Window.
+   *
+   * <b>NOTE:</b> This is purely an internal API and should not be used directly by users.
+   *
+   * @return the functions associated with this Window.
+   */
+  WindowFn<M, WK, WindowState<WV>, WindowOutput<WK, WV>> getInternalWindowFn();
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/windows/WindowFn.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/WindowFn.java b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowFn.java
new file mode 100644
index 0000000..8878bf9
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowFn.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.samza.operators.windows;
+
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.storage.kv.Entry;
+
+import java.util.function.BiFunction;
+
+
+/**
+ * Defines an internal representation of a window function.
+ *
+ * @param <M>  type of the input {@link MessageEnvelope} for the window
+ * @param <WK>  type of the window key in the output {@link MessageEnvelope}
+ * @param <WS>  type of the {@link WindowState} in the state store
+ * @param <WM>  type of the {@link MessageEnvelope} in the output stream
+ */
+public interface WindowFn<M extends MessageEnvelope, WK, WS extends WindowState, WM extends WindowOutput<WK, ?>> {
+
+  /**
+   * Get the transformation function of the {@link WindowFn}.
+   *
+   * @return  the transformation function which takes a {@link MessageEnvelope} of type {@code M} and its window state entry,
+   *          and transforms it to an {@link WindowOutput}
+   */
+  BiFunction<M, Entry<WK, WS>, WM> getTransformFn();
+
+  /**
+   * Get the state store functions for this {@link WindowFn}.
+   *
+   * @return  the state store functions
+   */
+  StoreFunctions<M, WK, WS> getStoreFns();
+
+  /**
+   * Get the trigger conditions for this {@link WindowFn}.
+   *
+   * @return  the trigger condition for this {@link WindowFn}
+   */
+  Trigger<M, WS> getTrigger();
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/windows/WindowOutput.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/WindowOutput.java b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowOutput.java
new file mode 100644
index 0000000..63e34c8
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowOutput.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.samza.operators.windows;
+
+import org.apache.samza.operators.data.MessageEnvelope;
+
+
+/**
+ * The type of output {@link MessageEnvelope}s in a window operator output stream.
+ *
+ * @param <K>  the type of key in the window output
+ * @param <M>  the type of value in the window output
+ */
+public final class WindowOutput<K, M> implements MessageEnvelope<K, M> {
+  private final K key;
+  private final M value;
+
+  WindowOutput(K key, M value) {
+    this.key = key;
+    this.value = value;
+  }
+
+  @Override public M getMessage() {
+    return this.value;
+  }
+
+  @Override public K getKey() {
+    return this.key;
+  }
+
+  static public <K, M> WindowOutput<K, M> of(K key, M result) {
+    return new WindowOutput<>(key, result);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/windows/WindowState.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/WindowState.java b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowState.java
new file mode 100644
index 0000000..835d749
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowState.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.samza.operators.windows;
+
+import org.apache.samza.annotation.InterfaceStability;
+
+
+/**
+ * This interface defines the methods a window state class has to implement. The programmers are allowed to implement
+ * customized window state to be stored in window state stores by implementing this interface class.
+ *
+ * @param <WV>  the type for window output value
+ */
+@InterfaceStability.Unstable
+public interface WindowState<WV> {
+  /**
+   * Method to get the system time when the first {@link org.apache.samza.operators.data.MessageEnvelope}
+   * in the window is received
+   *
+   * @return  nano-second of system time for the first {@link org.apache.samza.operators.data.MessageEnvelope}
+   *          received in the window
+   */
+  long getFirstMessageTimeNs();
+
+  /**
+   * Method to get the system time when the last {@link org.apache.samza.operators.data.MessageEnvelope}
+   * in the window is received
+   *
+   * @return  nano-second of system time for the last {@link org.apache.samza.operators.data.MessageEnvelope}
+   *          received in the window
+   */
+  long getLastMessageTimeNs();
+
+  /**
+   * Method to get the earliest event time in the window
+   *
+   * @return  the earliest event time in nano-second in the window
+   */
+  long getEarliestEventTimeNs();
+
+  /**
+   * Method to get the latest event time in the window
+   *
+   * @return  the latest event time in nano-second in the window
+   */
+  long getLatestEventTimeNs();
+
+  /**
+   * Method to get the total number of {@link org.apache.samza.operators.data.MessageEnvelope}s received in the window
+   *
+   * @return  number of {@link org.apache.samza.operators.data.MessageEnvelope}s in the window
+   */
+  long getNumberMessages();
+
+  /**
+   * Method to get the corresponding window's output value
+   *
+   * @return  the corresponding window's output value
+   */
+  WV getOutputValue();
+
+  /**
+   * Method to set the corresponding window's output value
+   *
+   * @param value  the corresponding window's output value
+   */
+  void setOutputValue(WV value);
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/windows/Windows.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/Windows.java b/samza-api/src/main/java/org/apache/samza/operators/windows/Windows.java
new file mode 100644
index 0000000..1a4ed8f
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/Windows.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.samza.operators.windows;
+
+import org.apache.samza.operators.data.MessageEnvelope;
+
+import java.util.Collection;
+import java.util.function.Function;
+
+
+/**
+ * This class defines a collection of {@link Window} functions. The public classes and methods here are intended to be
+ * used by the user (i.e. programmers) to create {@link Window} function directly.
+ *
+ */
+public final class Windows {
+
+  /**
+   * private constructor to prevent instantiation
+   */
+  private Windows() {}
+
+  static <M extends MessageEnvelope, WK, WV, WS extends WindowState<WV>, WM extends WindowOutput<WK, WV>> WindowFn<M, WK, WS, WM> getInternalWindowFn(
+      Window<M, WK, WV, WM> window) {
+    if (window instanceof SessionWindow) {
+      SessionWindow<M, WK, WV> sessionWindow = (SessionWindow<M, WK, WV>) window;
+      return (WindowFn<M, WK, WS, WM>) sessionWindow.getInternalWindowFn();
+    }
+    throw new IllegalArgumentException("Input window type not supported.");
+  }
+
+  /**
+   * Public static API methods start here
+   *
+   */
+
+  /**
+   * Static API method to create a {@link SessionWindow} in which the output value is simply the collection of input {@link MessageEnvelope}s
+   *
+   * @param sessionKeyFunction  function to calculate session window key
+   * @param <M>  type of input {@link MessageEnvelope}
+   * @param <WK>  type of the session window key
+   * @return  the {@link Window} function for the session
+   */
+  public static <M extends MessageEnvelope, WK> Window<M, WK, Collection<M>, WindowOutput<WK, Collection<M>>> intoSessions(Function<M, WK> sessionKeyFunction) {
+    return new SessionWindow<>(sessionKeyFunction, (m, c) -> {
+        c.add(m);
+        return c;
+      }
+    );
+  }
+
+  /**
+   * Static API method to create a {@link SessionWindow} in which the output value is a collection of {@code SI} from the input {@link MessageEnvelope}s
+   *
+   * @param sessionKeyFunction  function to calculate session window key
+   * @param sessionInfoExtractor  function to retrieve session info of type {@code SI} from the input {@link MessageEnvelope} of type {@code M}
+   * @param <M>  type of the input {@link MessageEnvelope}
+   * @param <WK>  type of the session window key
+   * @param <SI>  type of the session information retrieved from each input {@link MessageEnvelope} of type {@code M}
+   * @return  the {@link Window} function for the session
+   */
+  public static <M extends MessageEnvelope, WK, SI> Window<M, WK, Collection<SI>, WindowOutput<WK, Collection<SI>>> intoSessions(Function<M, WK> sessionKeyFunction,
+      Function<M, SI> sessionInfoExtractor) {
+    return new SessionWindow<>(sessionKeyFunction, (m, c) -> {
+        c.add(sessionInfoExtractor.apply(m));
+        return c;
+      }
+    );
+  }
+
+  /**
+   * Static API method to create a {@link SessionWindow} as a counter of input {@link MessageEnvelope}s
+   *
+   * @param sessionKeyFunction  function to calculate session window key
+   * @param <M>  type of the input {@link MessageEnvelope}
+   * @param <WK>  type of the session window key
+   * @return  the {@link Window} function for the session
+   */
+  public static <M extends MessageEnvelope, WK> Window<M, WK, Integer, WindowOutput<WK, Integer>> intoSessionCounter(Function<M, WK> sessionKeyFunction) {
+    return new SessionWindow<>(sessionKeyFunction, (m, c) -> c + 1);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/TestMessage.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/TestMessage.java b/samza-api/src/test/java/org/apache/samza/operators/TestMessage.java
deleted file mode 100644
index 8c56287..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/TestMessage.java
+++ /dev/null
@@ -1,47 +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.samza.operators;
-
-import org.apache.samza.operators.data.Message;
-
-
-public class TestMessage implements Message<String, String> {
-
-  private final String key;
-  private final String value;
-  private final long timestamp;
-
-  TestMessage(String key, String value, long timestamp) {
-    this.key = key;
-    this.value = value;
-    this.timestamp = timestamp;
-  }
-
-  @Override public String getMessage() {
-    return this.value;
-  }
-
-  @Override public String getKey() {
-    return this.key;
-  }
-
-  @Override public long getTimestamp() {
-    return this.timestamp;
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/TestMessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/TestMessageEnvelope.java b/samza-api/src/test/java/org/apache/samza/operators/TestMessageEnvelope.java
new file mode 100644
index 0000000..dfa69ac
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/operators/TestMessageEnvelope.java
@@ -0,0 +1,61 @@
+/*
+ * 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.samza.operators;
+
+import org.apache.samza.operators.data.MessageEnvelope;
+
+
+public class TestMessageEnvelope implements MessageEnvelope<String, TestMessageEnvelope.MessageType> {
+
+  private final String key;
+  private final MessageType value;
+
+  public TestMessageEnvelope(String key, String value, long eventTime) {
+    this.key = key;
+    this.value = new MessageType(value, eventTime);
+  }
+
+  @Override
+  public MessageType getMessage() {
+    return this.value;
+  }
+
+  @Override
+  public String getKey() {
+    return this.key;
+  }
+
+  public class MessageType {
+    private final String value;
+    private final long eventTime;
+
+    public MessageType(String value, long eventTime) {
+      this.value = value;
+      this.eventTime = eventTime;
+    }
+
+    public long getEventTime() {
+      return eventTime;
+    }
+
+    public String getValue() {
+      return value;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/TestMessageStream.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/TestMessageStream.java b/samza-api/src/test/java/org/apache/samza/operators/TestMessageStream.java
deleted file mode 100644
index 4dbe233..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/TestMessageStream.java
+++ /dev/null
@@ -1,180 +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.samza.operators;
-
-import org.apache.samza.operators.internal.Operators.*;
-import org.apache.samza.operators.internal.WindowOutput;
-import org.apache.samza.system.OutgoingMessageEnvelope;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-
-public class TestMessageStream {
-
-  @Test public void testMap() {
-    MessageStream<TestMessage> inputStream = new MessageStream<>();
-    Function<TestMessage, TestOutputMessage> xMap = m -> new TestOutputMessage(m.getKey(), m.getMessage().length() + 1, m.getTimestamp() + 2);
-    MessageStream<TestOutputMessage> outputStream = inputStream.map(xMap);
-    Collection<Operator> subs = inputStream.getSubscribers();
-    assertEquals(subs.size(), 1);
-    Operator<TestOutputMessage> mapOp = subs.iterator().next();
-    assertTrue(mapOp instanceof StreamOperator);
-    assertEquals(mapOp.getOutputStream(), outputStream);
-    // assert that the transformation function is what we defined above
-    TestMessage xTestMsg = mock(TestMessage.class);
-    when(xTestMsg.getKey()).thenReturn("test-msg-key");
-    when(xTestMsg.getMessage()).thenReturn("123456789");
-    when(xTestMsg.getTimestamp()).thenReturn(12345L);
-    Collection<TestOutputMessage> cOutputMsg = ((StreamOperator<TestMessage, TestOutputMessage>) mapOp).getFunction().apply(xTestMsg);
-    assertEquals(cOutputMsg.size(), 1);
-    TestOutputMessage outputMessage = cOutputMsg.iterator().next();
-    assertEquals(outputMessage.getKey(), xTestMsg.getKey());
-    assertEquals(outputMessage.getMessage(), Integer.valueOf(xTestMsg.getMessage().length() + 1));
-    assertEquals(outputMessage.getTimestamp(), xTestMsg.getTimestamp() + 2);
-  }
-
-  @Test public void testFlatMap() {
-    MessageStream<TestMessage> inputStream = new MessageStream<>();
-    Set<TestOutputMessage> flatOuts = new HashSet<TestOutputMessage>() { {
-        this.add(mock(TestOutputMessage.class));
-        this.add(mock(TestOutputMessage.class));
-        this.add(mock(TestOutputMessage.class));
-      } };
-    Function<TestMessage, Collection<TestOutputMessage>> xFlatMap = m -> flatOuts;
-    MessageStream<TestOutputMessage> outputStream = inputStream.flatMap(xFlatMap);
-    Collection<Operator> subs = inputStream.getSubscribers();
-    assertEquals(subs.size(), 1);
-    Operator<TestOutputMessage> flatMapOp = subs.iterator().next();
-    assertTrue(flatMapOp instanceof StreamOperator);
-    assertEquals(flatMapOp.getOutputStream(), outputStream);
-    // assert that the transformation function is what we defined above
-    assertEquals(((StreamOperator<TestMessage, TestOutputMessage>) flatMapOp).getFunction(), xFlatMap);
-  }
-
-  @Test public void testFilter() {
-    MessageStream<TestMessage> inputStream = new MessageStream<>();
-    Function<TestMessage, Boolean> xFilter = m -> m.getTimestamp() > 123456L;
-    MessageStream<TestMessage> outputStream = inputStream.filter(xFilter);
-    Collection<Operator> subs = inputStream.getSubscribers();
-    assertEquals(subs.size(), 1);
-    Operator<TestMessage> filterOp = subs.iterator().next();
-    assertTrue(filterOp instanceof StreamOperator);
-    assertEquals(filterOp.getOutputStream(), outputStream);
-    // assert that the transformation function is what we defined above
-    Function<TestMessage, Collection<TestMessage>> txfmFn = ((StreamOperator<TestMessage, TestMessage>) filterOp).getFunction();
-    TestMessage mockMsg = mock(TestMessage.class);
-    when(mockMsg.getTimestamp()).thenReturn(11111L);
-    Collection<TestMessage> output = txfmFn.apply(mockMsg);
-    assertTrue(output.isEmpty());
-    when(mockMsg.getTimestamp()).thenReturn(999999L);
-    output = txfmFn.apply(mockMsg);
-    assertEquals(output.size(), 1);
-    assertEquals(output.iterator().next(), mockMsg);
-  }
-
-  @Test public void testSink() {
-    MessageStream<TestMessage> inputStream = new MessageStream<>();
-    MessageStream.VoidFunction3<TestMessage, MessageCollector, TaskCoordinator> xSink = (m, mc, tc) -> {
-      mc.send(new OutgoingMessageEnvelope(new SystemStream("test-sys", "test-stream"), m.getMessage()));
-      tc.commit(TaskCoordinator.RequestScope.CURRENT_TASK);
-    };
-    inputStream.sink(xSink);
-    Collection<Operator> subs = inputStream.getSubscribers();
-    assertEquals(subs.size(), 1);
-    Operator<TestMessage> sinkOp = subs.iterator().next();
-    assertTrue(sinkOp instanceof SinkOperator);
-    assertEquals(((SinkOperator) sinkOp).getFunction(), xSink);
-    assertNull(((SinkOperator) sinkOp).getOutputStream());
-  }
-
-  @Test public void testWindow() {
-    MessageStream<TestMessage> inputStream = new MessageStream<>();
-    Windows.SessionWindow<TestMessage, String, Integer> window = mock(Windows.SessionWindow.class);
-    MessageStream<WindowOutput<String, Integer>> outStream = inputStream.window(window);
-    Collection<Operator> subs = inputStream.getSubscribers();
-    assertEquals(subs.size(), 1);
-    Operator<TestMessage> wndOp = subs.iterator().next();
-    assertTrue(wndOp instanceof WindowOperator);
-    assertEquals(((WindowOperator) wndOp).getOutputStream(), outStream);
-  }
-
-  @Test public void testJoin() {
-    MessageStream<TestMessage> source1 = new MessageStream<>();
-    MessageStream<TestMessage> source2 = new MessageStream<>();
-    BiFunction<TestMessage, TestMessage, TestOutputMessage> joiner = (m1, m2) -> new TestOutputMessage(m1.getKey(), m1.getMessage().length() + m2.getMessage().length(), m1.getTimestamp());
-    MessageStream<TestOutputMessage> joinOutput = source1.join(source2, joiner);
-    Collection<Operator> subs = source1.getSubscribers();
-    assertEquals(subs.size(), 1);
-    Operator<TestMessage> joinOp1 = subs.iterator().next();
-    assertTrue(joinOp1 instanceof PartialJoinOperator);
-    assertEquals(((PartialJoinOperator) joinOp1).getOutputStream(), joinOutput);
-    subs = source2.getSubscribers();
-    assertEquals(subs.size(), 1);
-    Operator<TestMessage> joinOp2 = subs.iterator().next();
-    assertTrue(joinOp2 instanceof PartialJoinOperator);
-    assertEquals(((PartialJoinOperator) joinOp2).getOutputStream(), joinOutput);
-    TestMessage joinMsg1 = new TestMessage("test-join-1", "join-msg-001", 11111L);
-    TestMessage joinMsg2 = new TestMessage("test-join-2", "join-msg-002", 22222L);
-    TestOutputMessage xOut = (TestOutputMessage) ((PartialJoinOperator) joinOp1).getFunction().apply(joinMsg1, joinMsg2);
-    assertEquals(xOut.getKey(), "test-join-1");
-    assertEquals(xOut.getMessage(), Integer.valueOf(24));
-    assertEquals(xOut.getTimestamp(), 11111L);
-    xOut = (TestOutputMessage) ((PartialJoinOperator) joinOp2).getFunction().apply(joinMsg2, joinMsg1);
-    assertEquals(xOut.getKey(), "test-join-1");
-    assertEquals(xOut.getMessage(), Integer.valueOf(24));
-    assertEquals(xOut.getTimestamp(), 11111L);
-  }
-
-  @Test public void testMerge() {
-    MessageStream<TestMessage> merge1 = new MessageStream<>();
-    Collection<MessageStream<TestMessage>> others = new ArrayList<MessageStream<TestMessage>>() { {
-        this.add(new MessageStream<>());
-        this.add(new MessageStream<>());
-      } };
-    MessageStream<TestMessage> mergeOutput = merge1.merge(others);
-    validateMergeOperator(merge1, mergeOutput);
-
-    others.forEach(merge -> validateMergeOperator(merge, mergeOutput));
-  }
-
-  private void validateMergeOperator(MessageStream<TestMessage> mergeSource, MessageStream<TestMessage> mergeOutput) {
-    Collection<Operator> subs = mergeSource.getSubscribers();
-    assertEquals(subs.size(), 1);
-    Operator<TestMessage> mergeOp = subs.iterator().next();
-    assertTrue(mergeOp instanceof StreamOperator);
-    assertEquals(((StreamOperator) mergeOp).getOutputStream(), mergeOutput);
-    TestMessage mockMsg = mock(TestMessage.class);
-    Collection<TestMessage> outputs = ((StreamOperator<TestMessage, TestMessage>) mergeOp).getFunction().apply(mockMsg);
-    assertEquals(outputs.size(), 1);
-    assertEquals(outputs.iterator().next(), mockMsg);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/TestMessageStreams.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/TestMessageStreams.java b/samza-api/src/test/java/org/apache/samza/operators/TestMessageStreams.java
deleted file mode 100644
index c5fcceb..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/TestMessageStreams.java
+++ /dev/null
@@ -1,35 +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.samza.operators;
-
-import org.apache.samza.Partition;
-import org.apache.samza.system.SystemStreamPartition;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-
-public class TestMessageStreams {
-
-  @Test public void testInput() {
-    SystemStreamPartition ssp = new SystemStreamPartition("my-system", "my-stream", new Partition(0));
-    MessageStreams.SystemMessageStream mSysStream = MessageStreams.input(ssp);
-    assertEquals(mSysStream.getSystemStreamPartition(), ssp);
-  }
-}


[6/6] samza git commit: SAMZA-1054: Refactor Operator APIs

Posted by ni...@apache.org.
SAMZA-1054: Refactor Operator APIs

Some suggestions for an Operator API refactor and misc. cleanup. It does contain some implementation changes, mostly due to deleted, extracted or merged classes. (e.g. OperatorFactory + ChainedOperators == OperatorImpls).

Since git marked several moved classes as (delete + new) instead, it's probably best to apply the diff locally and  browse the code in an IDE.

Some of the changes, in no particular order:
* Extracted XFunction interfaces into a .functions package in -api.
* -api's internal.Operators is now the -operators's spec.* package. Extracted interfaces and classes. Factory methods are now in OperatorSpecs.
* -api's MessageStreams is now -api's MessageStream interface and -operators's MessageStreamImpl.
* -api's internal.Windows classes are now in -api's .window package. Extracted interfaces and classes, but no implementation changes.
* OperatorFactory + ChainedOperators is now OperatorImpls, which is used from StreamOperatorAdaptorTask.
* Added a NoOpOperatorImpl, which acts as the root node for the OperatorImpl DAG returned by OperatorImpls.
* Removed usages of reactivestreams APIs since current code looks simpler without them. We can add them back when we need features like backpressure etc.
* Removed the InputSystemMessage interface.
* Made field names consistent (e.g Fn suffix for functions everywhere etc.).
* Some method/class visibility changes due to moved classes.
* General documentation changes, mostly to make public APIs clearer.

There are additional questions/tasks that we can address in future RBs:
* Updating Window and Trigger APIs.
* Merging samza-operator into samza-core.
* Questions about Message timestamp and Offset comparison semantics.
* Questions about OperatorSpec serialization (e.g. ID generation).
* Questions about StateStoreImpl and StoreFunctions.

Author: Prateek Maheshwari <pm...@linkedin.com>

Reviewers: Yi Pan <ni...@gmail.com>, Jagadish <ja...@gmail.com>

Closes #25 from prateekm/master


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

Branch: refs/heads/master
Commit: 00543804b3c32f1cbea0212e4a94e360b5a324cc
Parents: a980c96
Author: Prateek Maheshwari <pm...@linkedin.com>
Authored: Thu Dec 1 14:50:52 2016 -0800
Committer: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Committed: Thu Dec 1 14:50:52 2016 -0800

----------------------------------------------------------------------
 .../apache/samza/operators/MessageStream.java   | 197 +++-----
 .../apache/samza/operators/MessageStreams.java  |  81 ----
 .../samza/operators/StreamOperatorTask.java     |  51 ++
 .../apache/samza/operators/TriggerBuilder.java  | 323 -------------
 .../org/apache/samza/operators/WindowState.java |  81 ----
 .../org/apache/samza/operators/Windows.java     | 203 --------
 .../operators/data/IncomingSystemMessage.java   |  76 ---
 .../data/IncomingSystemMessageEnvelope.java     |  63 +++
 .../operators/data/InputSystemMessage.java      |  45 --
 .../apache/samza/operators/data/Message.java    |  64 ---
 .../samza/operators/data/MessageEnvelope.java   |  54 +++
 .../operators/functions/FilterFunction.java     |  40 ++
 .../operators/functions/FlatMapFunction.java    |  44 ++
 .../samza/operators/functions/JoinFunction.java |  44 ++
 .../samza/operators/functions/MapFunction.java  |  41 ++
 .../samza/operators/functions/SinkFunction.java |  46 ++
 .../samza/operators/internal/Operators.java     | 469 -------------------
 .../samza/operators/internal/Trigger.java       |  95 ----
 .../samza/operators/internal/WindowFn.java      |  60 ---
 .../samza/operators/internal/WindowOutput.java  |  55 ---
 .../operators/task/StreamOperatorTask.java      |  45 --
 .../samza/operators/windows/SessionWindow.java  | 102 ++++
 .../samza/operators/windows/StoreFunctions.java |  67 +++
 .../apache/samza/operators/windows/Trigger.java |  94 ++++
 .../samza/operators/windows/TriggerBuilder.java | 320 +++++++++++++
 .../apache/samza/operators/windows/Window.java  |  49 ++
 .../samza/operators/windows/WindowFn.java       |  59 +++
 .../samza/operators/windows/WindowOutput.java   |  51 ++
 .../samza/operators/windows/WindowState.java    |  85 ++++
 .../apache/samza/operators/windows/Windows.java | 100 ++++
 .../org/apache/samza/operators/TestMessage.java |  47 --
 .../samza/operators/TestMessageEnvelope.java    |  61 +++
 .../samza/operators/TestMessageStream.java      | 180 -------
 .../samza/operators/TestMessageStreams.java     |  35 --
 .../samza/operators/TestOutputMessage.java      |  47 --
 .../operators/TestOutputMessageEnvelope.java    |  43 ++
 .../samza/operators/TestTriggerBuilder.java     | 214 ---------
 .../org/apache/samza/operators/TestWindows.java | 106 -----
 .../data/TestIncomingSystemMessage.java         |   5 +-
 .../samza/operators/data/TestLongOffset.java    |   9 +-
 .../samza/operators/internal/TestOperators.java | 128 -----
 .../samza/operators/internal/TestTrigger.java   |  68 ---
 .../operators/internal/TestWindowOutput.java    |  36 --
 .../samza/operators/windows/TestTrigger.java    |  68 +++
 .../operators/windows/TestTriggerBuilder.java   | 226 +++++++++
 .../operators/windows/TestWindowOutput.java     |  36 ++
 .../samza/operators/windows/TestWindows.java    | 109 +++++
 .../samza/operators/MessageStreamImpl.java      | 134 ++++++
 .../apache/samza/operators/StateStoreImpl.java  |  56 +++
 .../operators/StreamOperatorAdaptorTask.java    | 105 +++++
 .../samza/operators/impl/ChainedOperators.java  | 119 -----
 .../samza/operators/impl/OperatorFactory.java   |  85 ----
 .../samza/operators/impl/OperatorImpl.java      |  76 +--
 .../samza/operators/impl/OperatorImpls.java     | 125 +++++
 .../operators/impl/PartialJoinOperatorImpl.java |  46 ++
 .../samza/operators/impl/ProcessorContext.java  |  53 ---
 .../samza/operators/impl/RootOperatorImpl.java  |  36 ++
 .../impl/SessionWindowOperatorImpl.java         |  67 +++
 .../operators/impl/SimpleOperatorImpl.java      |  49 --
 .../samza/operators/impl/SinkOperatorImpl.java  |  22 +-
 .../samza/operators/impl/StateStoreImpl.java    |  56 ---
 .../operators/impl/StreamOperatorImpl.java      |  47 ++
 .../operators/impl/join/PartialJoinOpImpl.java  |  44 --
 .../impl/window/SessionWindowImpl.java          |  65 ---
 .../samza/operators/spec/OperatorSpec.java      |  37 ++
 .../samza/operators/spec/OperatorSpecs.java     | 116 +++++
 .../operators/spec/PartialJoinOperatorSpec.java | 104 ++++
 .../samza/operators/spec/SinkOperatorSpec.java  |  62 +++
 .../operators/spec/StreamOperatorSpec.java      |  67 +++
 .../operators/spec/WindowOperatorSpec.java      | 119 +++++
 .../samza/task/StreamOperatorAdaptorTask.java   |  85 ----
 .../apache/samza/operators/BroadcastTask.java   | 101 ++++
 .../org/apache/samza/operators/JoinTask.java    |  77 +++
 .../operators/TestFluentStreamAdaptorTask.java  |  85 ++++
 .../samza/operators/TestFluentStreamTasks.java  | 112 +++++
 .../samza/operators/TestMessageStreamImpl.java  | 203 ++++++++
 .../samza/operators/TestStateStoreImpl.java     |  72 +++
 .../org/apache/samza/operators/WindowTask.java  |  70 +++
 .../data/JsonIncomingSystemMessageEnvelope.java |  60 +++
 .../operators/impl/TestChainedOperators.java    | 129 -----
 .../operators/impl/TestOperatorFactory.java     |  93 ----
 .../samza/operators/impl/TestOperatorImpl.java  |  48 +-
 .../samza/operators/impl/TestOperatorImpls.java | 183 ++++++++
 .../operators/impl/TestProcessorContext.java    |  40 --
 .../operators/impl/TestSessionWindowImpl.java   | 111 +++++
 .../operators/impl/TestSimpleOperatorImpl.java  |  55 ---
 .../operators/impl/TestSinkOperatorImpl.java    |  25 +-
 .../operators/impl/TestStateStoreImpl.java      |  69 ---
 .../operators/impl/TestStreamOperatorImpl.java  |  60 +++
 .../impl/window/TestSessionWindowImpl.java      | 105 -----
 .../samza/operators/spec/TestOperatorSpecs.java | 114 +++++
 .../samza/task/BroadcastOperatorTask.java       | 102 ----
 .../samza/task/InputJsonSystemMessage.java      |  67 ---
 .../org/apache/samza/task/JoinOperatorTask.java |  80 ----
 .../task/TestStreamOperatorAdaptorTask.java     |  80 ----
 .../samza/task/TestStreamOperatorTasks.java     | 105 -----
 .../apache/samza/task/WindowOperatorTask.java   |  71 ---
 97 files changed, 4374 insertions(+), 4240 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java
index dede631..d18536b 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java
@@ -16,173 +16,112 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package org.apache.samza.operators;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.internal.Operators;
-import org.apache.samza.operators.internal.Operators.Operator;
-import org.apache.samza.operators.internal.WindowOutput;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.functions.FilterFunction;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.functions.JoinFunction;
+import org.apache.samza.operators.functions.MapFunction;
+import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.operators.windows.Window;
+import org.apache.samza.operators.windows.WindowOutput;
+import org.apache.samza.operators.windows.WindowState;
 
-import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.function.BiFunction;
-import java.util.function.Function;
 
 
 /**
- * This class defines either the input or output streams to/from the operators. Users use the API methods defined here to
- * directly program the stream processing stages that processes a stream and generate another one.
+ * Represents a stream of {@link MessageEnvelope}s.
+ * <p>
+ * A {@link MessageStream} can be transformed into another {@link MessageStream} by applying the transforms in this API.
  *
- * @param <M>  Type of message in this stream
+ * @param <M>  type of {@link MessageEnvelope}s in this stream
  */
 @InterfaceStability.Unstable
-public class MessageStream<M extends Message> {
-
-  private final Set<Operator> subscribers = new HashSet<>();
-
-  /**
-   * Helper method to get the corresponding list of subscribers to a specific {@link MessageStream}.
-   *
-   * NOTE: This is purely an internal API and should not be used directly by programmers.
-   *
-   * @return A unmodifiable set containing all {@link Operator}s that subscribe to this {@link MessageStream} object
-   */
-  public Collection<Operator> getSubscribers() {
-    return Collections.unmodifiableSet(this.subscribers);
-  }
-
-  /**
-   * Public API methods start here
-   */
+public interface MessageStream<M extends MessageEnvelope> {
 
   /**
-   * Defines a function API that takes three input parameters w/ types {@code A}, {@code B}, and {@code C} and w/o a return value
+   * Applies the provided 1:1 {@link MapFunction} to {@link MessageEnvelope}s in this {@link MessageStream} and returns the
+   * transformed {@link MessageStream}.
    *
-   * @param <A>  the type of input {@code a}
-   * @param <B>  the type of input {@code b}
-   * @param <C>  the type of input {@code c}
+   * @param mapFn  the function to transform a {@link MessageEnvelope} to another {@link MessageEnvelope}
+   * @param <TM>  the type of {@link MessageEnvelope}s in the transformed {@link MessageStream}
+   * @return the transformed {@link MessageStream}
    */
-  @FunctionalInterface
-  public interface VoidFunction3<A, B, C> {
-    public void apply(A a, B b, C c);
-  }
+  <TM extends MessageEnvelope> MessageStream<TM> map(MapFunction<M, TM> mapFn);
 
   /**
-   * Method to apply a map function (1:1) on a {@link MessageStream}
+   * Applies the provided 1:n {@link FlatMapFunction} to transform a {@link MessageEnvelope} in this {@link MessageStream}
+   * to n {@link MessageEnvelope}s in the transformed {@link MessageStream}
    *
-   * @param mapper  the mapper function to map one input {@link Message} to one output {@link Message}
-   * @param <OM>  the type of the output {@link Message} in the output {@link MessageStream}
-   * @return the output {@link MessageStream} by applying the map function on the input {@link MessageStream}
+   * @param flatMapFn  the function to transform a {@link MessageEnvelope} to zero or more {@link MessageEnvelope}s
+   * @param <TM>  the type of {@link MessageEnvelope}s in the transformed {@link MessageStream}
+   * @return the transformed {@link MessageStream}
    */
-  public <OM extends Message> MessageStream<OM> map(Function<M, OM> mapper) {
-    Operator<OM> op = Operators.<M, OM>getStreamOperator(m -> new ArrayList<OM>() { {
-        OM r = mapper.apply(m);
-        if (r != null) {
-          this.add(r);
-        }
-      } });
-    this.subscribers.add(op);
-    return op.getOutputStream();
-  }
+  <TM extends MessageEnvelope> MessageStream<TM> flatMap(FlatMapFunction<M, TM> flatMapFn);
 
   /**
-   * Method to apply a flatMap function (1:n) on a {@link MessageStream}
+   * Applies the provided {@link FilterFunction} to {@link MessageEnvelope}s in this {@link MessageStream} and returns the
+   * transformed {@link MessageStream}.
+   * <p>
+   * The {@link FilterFunction} is a predicate which determines whether a {@link MessageEnvelope} in this {@link MessageStream}
+   * should be retained in the transformed {@link MessageStream}.
    *
-   * @param flatMapper  the flat mapper function to map one input {@link Message} to zero or more output {@link Message}s
-   * @param <OM>  the type of the output {@link Message} in the output {@link MessageStream}
-   * @return the output {@link MessageStream} by applying the map function on the input {@link MessageStream}
+   * @param filterFn  the predicate to filter {@link MessageEnvelope}s from this {@link MessageStream}
+   * @return the transformed {@link MessageStream}
    */
-  public <OM extends Message> MessageStream<OM> flatMap(Function<M, Collection<OM>> flatMapper) {
-    Operator<OM> op = Operators.getStreamOperator(flatMapper);
-    this.subscribers.add(op);
-    return op.getOutputStream();
-  }
+  MessageStream<M> filter(FilterFunction<M> filterFn);
 
   /**
-   * Method to apply a filter function on a {@link MessageStream}
+   * Allows sending {@link MessageEnvelope}s in this {@link MessageStream} to an output
+   * {@link org.apache.samza.system.SystemStream} using the provided {@link SinkFunction}.
    *
-   * @param filter  the filter function to filter input {@link Message}s from the input {@link MessageStream}
-   * @return the output {@link MessageStream} after applying the filter function on the input {@link MessageStream}
+   * @param sinkFn  the function to send {@link MessageEnvelope}s in this stream to output systems
    */
-  public MessageStream<M> filter(Function<M, Boolean> filter) {
-    Operator<M> op = Operators.<M, M>getStreamOperator(t -> new ArrayList<M>() { {
-        if (filter.apply(t)) {
-          this.add(t);
-        }
-      } });
-    this.subscribers.add(op);
-    return op.getOutputStream();
-  }
+  void sink(SinkFunction<M> sinkFn);
 
   /**
-   * Method to send an input {@link MessageStream} to an output {@link org.apache.samza.system.SystemStream}, and allows the output {@link MessageStream}
-   * to be consumed by downstream stream operators again.
+   * Groups the {@link MessageEnvelope}s in this {@link MessageStream} according to the provided {@link Window} semantics
+   * (e.g. tumbling, sliding or session windows) and returns the transformed {@link MessageStream} of
+   * {@link WindowOutput}s.
+   * <p>
+   * Use the {@link org.apache.samza.operators.windows.Windows} helper methods to create the appropriate windows.
    *
-   * @param sink  the user-defined sink function to send the input {@link Message}s to the external output systems
+   * @param window  the {@link Window} to group and process {@link MessageEnvelope}s from this {@link MessageStream}
+   * @param <WK>  the type of key in the {@link WindowOutput} from the {@link Window}
+   * @param <WV>  the type of value in the {@link WindowOutput} from the {@link Window}
+   * @param <WS>  the type of window state kept in the {@link Window}
+   * @param <WM>  the type of {@link WindowOutput} in the transformed {@link MessageStream}
+   * @return  the transformed {@link MessageStream}
    */
-  public void sink(VoidFunction3<M, MessageCollector, TaskCoordinator> sink) {
-    this.subscribers.add(Operators.getSinkOperator(sink));
-  }
+  <WK, WV, WS extends WindowState<WV>, WM extends WindowOutput<WK, WV>> MessageStream<WM> window(
+      Window<M, WK, WV, WM> window);
 
   /**
-   * Method to perform a window function (i.e. a group-by, aggregate function) on a {@link MessageStream}
+   * Joins this {@link MessageStream} with another {@link MessageStream} using the provided pairwise {@link JoinFunction}.
+   * <p>
+   * We currently only support 2-way joins.
    *
-   * @param window  the window function to group and aggregate the input {@link Message}s from the input {@link MessageStream}
-   * @param <WK>  the type of key in the output {@link Message} from the {@link Windows.Window} function
-   * @param <WV>  the type of output value from
-   * @param <WS>  the type of window state kept in the {@link Windows.Window} function
-   * @param <WM>  the type of {@link org.apache.samza.operators.internal.WindowOutput} message from the {@link Windows.Window} function
-   * @return the output {@link MessageStream} after applying the window function on the input {@link MessageStream}
-   */
-  public <WK, WV, WS extends WindowState<WV>, WM extends WindowOutput<WK, WV>> MessageStream<WM> window(Windows.Window<M, WK, WV, WM> window) {
-    Operator<WM> wndOp = Operators.getWindowOperator(Windows.getInternalWindowFn(window));
-    this.subscribers.add(wndOp);
-    return wndOp.getOutputStream();
-  }
-
-  /**
-   * Method to add an input {@link MessageStream} to a join function. Note that we currently only support 2-way joins.
-   *
-   * @param other  the other stream to be joined w/
-   * @param merger  the common function to merge messages from this {@link MessageStream} and {@code other}
+   * @param otherStream  the other {@link MessageStream} to be joined with
+   * @param joinFn  the function to join {@link MessageEnvelope}s from this and the other {@link MessageStream}
    * @param <K>  the type of join key
-   * @param <JM>  the type of message in the {@link Message} from the other join stream
-   * @param <RM>  the type of message in the {@link Message} from the join function
-   * @return the output {@link MessageStream} from the join function {@code joiner}
+   * @param <OM>  the type of {@link MessageEnvelope}s in the other stream
+   * @param <RM>  the type of {@link MessageEnvelope}s resulting from the {@code joinFn}
+   * @return  the joined {@link MessageStream}
    */
-  public <K, JM extends Message<K, ?>, RM extends Message> MessageStream<RM> join(MessageStream<JM> other,
-      BiFunction<M, JM, RM> merger) {
-    MessageStream<RM> outputStream = new MessageStream<>();
-
-    BiFunction<M, JM, RM> parJoin1 = merger::apply;
-    BiFunction<JM, M, RM> parJoin2 = (m, t1) -> merger.apply(t1, m);
-
-    // TODO: need to add default store functions for the two partial join functions
-
-    other.subscribers.add(Operators.<JM, K, M, RM>getPartialJoinOperator(parJoin2, outputStream));
-    this.subscribers.add(Operators.<M, K, JM, RM>getPartialJoinOperator(parJoin1, outputStream));
-    return outputStream;
-  }
+  <K, OM extends MessageEnvelope<K, ?>, RM extends MessageEnvelope> MessageStream<RM> join(MessageStream<OM> otherStream,
+      JoinFunction<M, OM, RM> joinFn);
 
   /**
-   * Method to merge all {@code others} streams w/ this {@link MessageStream}. The merging streams must have the same type {@code M}
+   * Merge all {@code otherStreams} with this {@link MessageStream}.
+   * <p>
+   * The merging streams must have the same {@link MessageEnvelope} type {@code M}.
    *
-   * @param others  other streams to be merged w/ this one
-   * @return  the merged output stream
+   * @param otherStreams  other {@link MessageStream}s to be merged with this {@link MessageStream}
+   * @return  the merged {@link MessageStream}
    */
-  public MessageStream<M> merge(Collection<MessageStream<M>> others) {
-    MessageStream<M> outputStream = new MessageStream<>();
-
-    others.add(this);
-    others.forEach(other -> other.subscribers.add(Operators.getMergeOperator(outputStream)));
-    return outputStream;
-  }
-
+  MessageStream<M> merge(Collection<MessageStream<M>> otherStreams);
+  
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/MessageStreams.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/MessageStreams.java b/samza-api/src/main/java/org/apache/samza/operators/MessageStreams.java
deleted file mode 100644
index 51bf482..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/MessageStreams.java
+++ /dev/null
@@ -1,81 +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.samza.operators;
-
-import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.IncomingSystemMessage;
-import org.apache.samza.system.SystemStreamPartition;
-
-
-/**
- * This class defines all methods to create a {@link MessageStream} object. Users can use this to create an {@link MessageStream}
- * from a specific input source.
- *
- */
-@InterfaceStability.Unstable
-public final class MessageStreams {
-
-  /**
-   * private constructor to prevent instantiation
-   */
-  private MessageStreams() {}
-
-  /**
-   * private class for system input/output {@link MessageStream}
-   */
-  public static final class SystemMessageStream extends MessageStream<IncomingSystemMessage> {
-    /**
-     * The corresponding {@link org.apache.samza.system.SystemStream}
-     */
-    private final SystemStreamPartition ssp;
-
-    /**
-     * Constructor for input system stream
-     *
-     * @param ssp  the input {@link SystemStreamPartition} for the input {@link SystemMessageStream}
-     */
-    private SystemMessageStream(SystemStreamPartition ssp) {
-      this.ssp = ssp;
-    }
-
-    /**
-     * Getter for the {@link SystemStreamPartition} of the input
-     *
-     * @return the input {@link SystemStreamPartition}
-     */
-    public SystemStreamPartition getSystemStreamPartition() {
-      return this.ssp;
-    }
-  }
-
-  /**
-   * Public static API methods start here
-   */
-
-  /**
-   * Static API method to create a {@link MessageStream} from a system input stream
-   *
-   * @param ssp  the input {@link SystemStreamPartition}
-   * @return the {@link MessageStream} object takes {@code ssp} as the input
-   */
-  public static SystemMessageStream input(SystemStreamPartition ssp) {
-    return new SystemMessageStream(ssp);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/StreamOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/StreamOperatorTask.java b/samza-api/src/main/java/org/apache/samza/operators/StreamOperatorTask.java
new file mode 100644
index 0000000..16cf27a
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/StreamOperatorTask.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.samza.operators;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.operators.data.IncomingSystemMessageEnvelope;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.util.Map;
+
+
+/**
+ * A {@link StreamOperatorTask} is the basic interface to implement for processing {@link MessageStream}s.
+ * Implementations can describe the transformation steps for each {@link MessageStream} in the
+ * {@link #transform} method using {@link MessageStream} APIs.
+ * <p>
+ * Implementations may be augmented by implementing {@link org.apache.samza.task.InitableTask},
+ * {@link org.apache.samza.task.WindowableTask} and {@link org.apache.samza.task.ClosableTask} interfaces,
+ * but should not implement {@link org.apache.samza.task.StreamTask} or {@link org.apache.samza.task.AsyncStreamTask}
+ * interfaces.
+ */
+@InterfaceStability.Unstable
+public interface StreamOperatorTask {
+
+  /**
+   * Describe the transformation steps for each {@link MessageStream}s for this task using the
+   * {@link MessageStream} APIs. Each {@link MessageStream} corresponds to one {@link SystemStreamPartition}
+   * in the input system.
+   *
+   * @param messageStreams the {@link MessageStream}s that receive {@link IncomingSystemMessageEnvelope}s
+   *                       from their corresponding {@link org.apache.samza.system.SystemStreamPartition}
+   */
+  void transform(Map<SystemStreamPartition, MessageStream<IncomingSystemMessageEnvelope>> messageStreams);
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/TriggerBuilder.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/TriggerBuilder.java b/samza-api/src/main/java/org/apache/samza/operators/TriggerBuilder.java
deleted file mode 100644
index 5b3f4d0..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/TriggerBuilder.java
+++ /dev/null
@@ -1,323 +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.samza.operators;
-
-
-import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.internal.Trigger;
-
-import java.util.concurrent.TimeUnit;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-
-
-/**
- * This class defines a builder of {@link org.apache.samza.operators.internal.Trigger} object for a {@link Windows.Window}. The triggers are categorized into
- * three types:
- *
- * <p>
- *   early trigger: defines the condition when the first output from the window function is sent.
- *   late trigger: defines the condition when the updated output after the first output is sent.
- *   timer trigger: defines a system timeout condition to trigger output if no more inputs are received to enable early/late triggers
- * </p>
- *
- * If multiple conditions are defined for a specific type of trigger, the aggregated trigger is the disjunction of the each individual trigger (i.e. OR).
- *
- * NOTE: Programmers should not use classes defined in {@link org.apache.samza.operators.internal} to create triggers
- *
- *
- * @param <M>  the type of input {@link Message} to the {@link Windows.Window}
- * @param <V>  the type of output value from the {@link Windows.Window}
- */
-@InterfaceStability.Unstable
-public final class TriggerBuilder<M extends Message, V> {
-
-  /**
-   * Predicate helper to OR multiple trigger conditions
-   */
-  static class PredicateHelper {
-    static <M, S> BiFunction<M, S, Boolean> or(BiFunction<M, S, Boolean> lhs, BiFunction<M, S, Boolean> rhs) {
-      return (m, s) -> lhs.apply(m, s) || rhs.apply(m, s);
-    }
-
-    static <S> Function<S, Boolean> or(Function<S, Boolean> lhs, Function<S, Boolean> rhs) {
-      return s -> lhs.apply(s) || rhs.apply(s);
-    }
-  }
-
-  /**
-   * The early trigger condition that determines the first output from the {@link Windows.Window}
-   */
-  private BiFunction<M, WindowState<V>, Boolean> earlyTrigger = null;
-
-  /**
-   * The late trigger condition that determines the late output(s) from the {@link Windows.Window}
-   */
-  private BiFunction<M, WindowState<V>, Boolean> lateTrigger = null;
-
-  /**
-   * The system timer based trigger conditions that guarantees the {@link Windows.Window} proceeds forward
-   */
-  private Function<WindowState<V>, Boolean> timerTrigger = null;
-
-  /**
-   * The state updater function to be applied after the first output is triggered
-   */
-  private Function<WindowState<V>, WindowState<V>> earlyTriggerUpdater = Function.identity();
-
-  /**
-   * The state updater function to be applied after the late output is triggered
-   */
-  private Function<WindowState<V>, WindowState<V>> lateTriggerUpdater = Function.identity();
-
-  /**
-   * Helper method to add a trigger condition
-   *
-   * @param currentTrigger  current trigger condition
-   * @param newTrigger  new trigger condition
-   * @return  combined trigger condition that is {@code currentTrigger} OR {@code newTrigger}
-   */
-  private BiFunction<M, WindowState<V>, Boolean> addTrigger(BiFunction<M, WindowState<V>, Boolean> currentTrigger,
-      BiFunction<M, WindowState<V>, Boolean> newTrigger) {
-    if (currentTrigger == null) {
-      return newTrigger;
-    }
-
-    return PredicateHelper.or(currentTrigger, newTrigger);
-  }
-
-  /**
-   * Helper method to add a system timer trigger
-   *
-   * @param currentTimer  current timer condition
-   * @param newTimer  new timer condition
-   * @return  combined timer condition that is {@code currentTimer} OR {@code newTimer}
-   */
-  private Function<WindowState<V>, Boolean> addTimerTrigger(Function<WindowState<V>, Boolean> currentTimer,
-      Function<WindowState<V>, Boolean> newTimer) {
-    if (currentTimer == null) {
-      return newTimer;
-    }
-
-    return PredicateHelper.or(currentTimer, newTimer);
-  }
-
-  /**
-   * default constructor to prevent instantiation
-   */
-  private TriggerBuilder() {}
-
-  /**
-   * Constructor that set the size limit as the early trigger for a window
-   *
-   * @param sizeLimit  the number of messages in a window that would trigger the first output
-   */
-  private TriggerBuilder(long sizeLimit) {
-    this.earlyTrigger = (m, s) -> s.getNumberMessages() > sizeLimit;
-  }
-
-  /**
-   * Constructor that set the event time length as the early trigger
-   *
-   * @param eventTimeFunction  the function that calculate the event time in nano-second from the input {@link Message}
-   * @param wndLenMs  the window length in event time in milli-second
-   */
-  private TriggerBuilder(Function<M, Long> eventTimeFunction, long wndLenMs) {
-    this.earlyTrigger = (m, s) ->
-        TimeUnit.NANOSECONDS.toMillis(Math.max(s.getLatestEventTimeNs() - s.getEarliestEventTimeNs(),
-            eventTimeFunction.apply(m) - s.getEarliestEventTimeNs())) > wndLenMs;
-  }
-
-  /**
-   * Constructor that set the special token message as the early trigger
-   *
-   * @param tokenFunc  the function that checks whether an input {@link Message} is a token message that triggers window output
-   */
-  private TriggerBuilder(Function<M, Boolean> tokenFunc) {
-    this.earlyTrigger = (m, s) -> tokenFunc.apply(m);
-  }
-
-  /**
-   * Build method that creates an {@link org.apache.samza.operators.internal.Trigger} object based on the trigger conditions set in {@link TriggerBuilder}
-   * This is kept package private and only used by {@link Windows} to convert the mutable {@link TriggerBuilder} object to an immutable {@link Trigger} object
-   *
-   * @return  the final {@link org.apache.samza.operators.internal.Trigger} object
-   */
-  Trigger<M, WindowState<V>> build() {
-    return Trigger.createTrigger(this.timerTrigger, this.earlyTrigger, this.lateTrigger, this.earlyTriggerUpdater, this.lateTriggerUpdater);
-  }
-
-  /**
-   * Public API methods start here
-   */
-
-
-  /**
-   * API method to allow users to set an update method to update the output value after the first window output is triggered
-   * by the early trigger condition
-   *
-   * @param onTriggerFunc  the method to update the output value after the early trigger
-   * @return  the {@link TriggerBuilder} object
-   */
-  public TriggerBuilder<M, V> onEarlyTrigger(Function<V, V> onTriggerFunc) {
-    this.earlyTriggerUpdater = s -> {
-      s.setOutputValue(onTriggerFunc.apply(s.getOutputValue()));
-      return s;
-    };
-    return this;
-  }
-
-  /**
-   * API method to allow users to set an update method to update the output value after a late window output is triggered
-   * by the late trigger condition
-   *
-   * @param onTriggerFunc  the method to update the output value after the late trigger
-   * @return  the {@link TriggerBuilder} object
-   */
-  public TriggerBuilder<M, V> onLateTrigger(Function<V, V> onTriggerFunc) {
-    this.lateTriggerUpdater = s -> {
-      s.setOutputValue(onTriggerFunc.apply(s.getOutputValue()));
-      return s;
-    };
-    return this;
-  }
-
-  /**
-   * API method to allow users to add a system timer trigger based on timeout after the last message received in the window
-   *
-   * @param timeoutMs  the timeout in ms after the last message received in the window
-   * @return  the {@link TriggerBuilder} object
-   */
-  public TriggerBuilder<M, V> addTimeoutSinceLastMessage(long timeoutMs) {
-    this.timerTrigger = this.addTimerTrigger(this.timerTrigger,
-        s -> TimeUnit.NANOSECONDS.toMillis(s.getLastMessageTimeNs()) + timeoutMs < System.currentTimeMillis());
-    return this;
-  }
-
-  /**
-   * API method to allow users to add a system timer trigger based on the timeout after the first message received in the window
-   *
-   * @param timeoutMs  the timeout in ms after the first message received in the window
-   * @return  the {@link TriggerBuilder} object
-   */
-  public TriggerBuilder<M, V> addTimeoutSinceFirstMessage(long timeoutMs) {
-    this.timerTrigger = this.addTimerTrigger(this.timerTrigger, s ->
-        TimeUnit.NANOSECONDS.toMillis(s.getFirstMessageTimeNs()) + timeoutMs < System.currentTimeMillis());
-    return this;
-  }
-
-  /**
-   * API method allow users to add a late trigger based on the window size limit
-   *
-   * @param sizeLimit  limit on the number of messages in window
-   * @return  the {@link TriggerBuilder} object
-   */
-  public TriggerBuilder<M, V> addLateTriggerOnSizeLimit(long sizeLimit) {
-    this.lateTrigger = this.addTrigger(this.lateTrigger, (m, s) -> s.getNumberMessages() > sizeLimit);
-    return this;
-  }
-
-  /**
-   * API method to allow users to define a customized late trigger function based on input message and the window state
-   *
-   * @param lateTrigger  the late trigger condition based on input {@link Message} and the current {@link WindowState}
-   * @return  the {@link TriggerBuilder} object
-   */
-  public TriggerBuilder<M, V> addLateTrigger(BiFunction<M, WindowState<V>, Boolean> lateTrigger) {
-    this.lateTrigger = this.addTrigger(this.lateTrigger, lateTrigger);
-    return this;
-  }
-
-  /**
-   * Static API method to create a {@link TriggerBuilder} w/ early trigger condition based on window size limit
-   *
-   * @param sizeLimit  window size limit
-   * @param <M>  the type of input {@link Message}
-   * @param <V>  the type of {@link Windows.Window} output value
-   * @return  the {@link TriggerBuilder} object
-   */
-  public static <M extends Message, V> TriggerBuilder<M, V> earlyTriggerWhenExceedWndLen(long sizeLimit) {
-    return new TriggerBuilder<M, V>(sizeLimit);
-  }
-
-  /**
-   * Static API method to create a {@link TriggerBuilder} w/ early trigger condition based on event time window
-   *
-   *
-   * @param eventTimeFunc  the function to get the event time from the input message
-   * @param eventTimeWndSizeMs  the event time window size in Ms
-   * @param <M>  the type of input {@link Message}
-   * @param <V>  the type of {@link Windows.Window} output value
-   * @return  the {@link TriggerBuilder} object
-   */
-  public static <M extends Message, V> TriggerBuilder<M, V> earlyTriggerOnEventTime(Function<M, Long> eventTimeFunc, long eventTimeWndSizeMs) {
-    return new TriggerBuilder<M, V>(eventTimeFunc, eventTimeWndSizeMs);
-  }
-
-  /**
-   * Static API method to create a {@link TriggerBuilder} w/ early trigger condition based on token messages
-   *
-   * @param tokenFunc  the function to determine whether an input message is a window token or not
-   * @param <M>  the type of input {@link Message}
-   * @param <V>  the type of {@link Windows.Window} output value
-   * @return  the {@link TriggerBuilder} object
-   */
-  public static <M extends Message, V> TriggerBuilder<M, V> earlyTriggerOnTokenMsg(Function<M, Boolean> tokenFunc) {
-    return new TriggerBuilder<M, V>(tokenFunc);
-  }
-
-  /**
-   * Static API method to allow customized early trigger condition based on input {@link Message} and the corresponding {@link WindowState}
-   *
-   * @param earlyTrigger  the user defined early trigger condition
-   * @param <M>   the input message type
-   * @param <V>   the output value from the window
-   * @return   the {@link TriggerBuilder} object
-   */
-  public static <M extends Message, V> TriggerBuilder<M, V> earlyTrigger(BiFunction<M, WindowState<V>, Boolean> earlyTrigger) {
-    TriggerBuilder<M, V> newTriggers =  new TriggerBuilder<M, V>();
-    newTriggers.earlyTrigger = newTriggers.addTrigger(newTriggers.earlyTrigger, earlyTrigger);
-    return newTriggers;
-  }
-
-  /**
-   * Static API method to create a {@link TriggerBuilder} w/ system timeout after the last message received in the window
-   *
-   * @param timeoutMs  timeout in ms after the last message received
-   * @param <M>  the type of input {@link Message}
-   * @param <V>  the type of {@link Windows.Window} output value
-   * @return  the {@link TriggerBuilder} object
-   */
-  public static <M extends Message, V> TriggerBuilder<M, V> timeoutSinceLastMessage(long timeoutMs) {
-    return new TriggerBuilder<M, V>().addTimeoutSinceLastMessage(timeoutMs);
-  }
-
-  /**
-   * Static API method to create a {@link TriggerBuilder} w/ system timeout after the first message received in the window
-   *
-   * @param timeoutMs  timeout in ms after the first message received
-   * @param <M>  the type of input {@link Message}
-   * @param <V>  the type of {@link Windows.Window} output value
-   * @return  the {@link TriggerBuilder} object
-   */
-  public static <M extends Message, V> TriggerBuilder<M, V> timeoutSinceFirstMessage(long timeoutMs) {
-    return new TriggerBuilder<M, V>().addTimeoutSinceFirstMessage(timeoutMs);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/WindowState.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/WindowState.java b/samza-api/src/main/java/org/apache/samza/operators/WindowState.java
deleted file mode 100644
index 759f2d8..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/WindowState.java
+++ /dev/null
@@ -1,81 +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.samza.operators;
-
-import org.apache.samza.annotation.InterfaceStability;
-
-
-/**
- * This interface defines the methods a window state class has to implement. The programmers are allowed to implement
- * customized window state to be stored in window state stores by implementing this interface class.
- *
- * @param <WV>  the type for window output value
- */
-@InterfaceStability.Unstable
-public interface WindowState<WV> {
-  /**
-   * Method to get the system time when the first message in the window is received
-   *
-   * @return  nano-second of system time for the first message received in the window
-   */
-  long getFirstMessageTimeNs();
-
-  /**
-   * Method to get the system time when the last message in the window is received
-   *
-   * @return  nano-second of system time for the last message received in the window
-   */
-  long getLastMessageTimeNs();
-
-  /**
-   * Method to get the earliest event time in the window
-   *
-   * @return  the earliest event time in nano-second in the window
-   */
-  long getEarliestEventTimeNs();
-
-  /**
-   * Method to get the latest event time in the window
-   *
-   * @return  the latest event time in nano-second in the window
-   */
-  long getLatestEventTimeNs();
-
-  /**
-   * Method to get the total number of messages received in the window
-   *
-   * @return  number of messages in the window
-   */
-  long getNumberMessages();
-
-  /**
-   * Method to get the corresponding window's output value
-   *
-   * @return  the corresponding window's output value
-   */
-  WV getOutputValue();
-
-  /**
-   * Method to set the corresponding window's output value
-   *
-   * @param value  the corresponding window's output value
-   */
-  void setOutputValue(WV value);
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/Windows.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/Windows.java b/samza-api/src/main/java/org/apache/samza/operators/Windows.java
deleted file mode 100644
index 6619f41..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/Windows.java
+++ /dev/null
@@ -1,203 +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.samza.operators;
-
-import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.internal.Operators;
-import org.apache.samza.operators.internal.Trigger;
-import org.apache.samza.operators.internal.WindowFn;
-import org.apache.samza.operators.internal.WindowOutput;
-import org.apache.samza.storage.kv.Entry;
-
-import java.util.Collection;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-
-
-/**
- * This class defines a collection of {@link Window} functions. The public classes and methods here are intended to be
- * used by the user (i.e. programmers) to create {@link Window} function directly.
- *
- */
-@InterfaceStability.Unstable
-public final class Windows {
-
-  /**
-   * private constructor to prevent instantiation
-   */
-  private Windows() {}
-
-  /**
-   * This class defines a session window function class
-   *
-   * @param <M>  the type of input {@link Message}
-   * @param <WK>  the type of session key in the session window
-   * @param <WV>  the type of output value in each session window
-   */
-  static class SessionWindow<M extends Message, WK, WV> implements Window<M, WK, WV, WindowOutput<WK, WV>> {
-
-    /**
-     * Constructor. Made private s.t. it can only be instantiated via the static API methods in {@link Windows}
-     *
-     * @param sessionKeyFunction  function to get the session key from the input {@link Message}
-     * @param aggregator  function to calculate the output value based on the input {@link Message} and current output value
-     */
-    private SessionWindow(Function<M, WK> sessionKeyFunction, BiFunction<M, WV, WV> aggregator) {
-      this.wndKeyFunction = sessionKeyFunction;
-      this.aggregator = aggregator;
-    }
-
-    /**
-     * function to calculate the window key from input message
-     */
-    private final Function<M, WK> wndKeyFunction;
-
-    /**
-     * function to calculate the output value from the input message and the current output value
-     */
-    private final BiFunction<M, WV, WV> aggregator;
-
-    /**
-     * trigger condition that determines when to send out the output value in a {@link WindowOutput} message
-     */
-    private Trigger<M, WindowState<WV>> trigger = null;
-
-    //TODO: need to create a set of {@link StoreFunctions} that is default to input {@link Message} type for {@link Window}
-    private Operators.StoreFunctions<M, WK, WindowState<WV>> storeFunctions = null;
-
-    /**
-     * Public API methods start here
-     */
-
-    /**
-     * Public API method to define the watermark trigger for the window operator
-     *
-     * @param wndTrigger {@link Trigger} function defines the watermark trigger for this {@link SessionWindow}
-     * @return The window operator w/ the defined watermark trigger
-     */
-    @Override
-    public Window<M, WK, WV, WindowOutput<WK, WV>> setTriggers(TriggerBuilder<M, WV> wndTrigger) {
-      this.trigger = wndTrigger.build();
-      return this;
-    }
-
-    private BiFunction<M, Entry<WK, WindowState<WV>>, WindowOutput<WK, WV>> getTransformFunc() {
-      // TODO: actual implementation of the main session window logic, based on the wndKeyFunction, aggregator, and triggers;
-      return null;
-    }
-
-    private WindowFn<M, WK, WindowState<WV>, WindowOutput<WK, WV>> getInternalWindowFn() {
-      return new WindowFn<M, WK, WindowState<WV>, WindowOutput<WK, WV>>() {
-
-        @Override public BiFunction<M, Entry<WK, WindowState<WV>>, WindowOutput<WK, WV>> getTransformFunc() {
-          return SessionWindow.this.getTransformFunc();
-        }
-
-        @Override public Operators.StoreFunctions<M, WK, WindowState<WV>> getStoreFuncs() {
-          return SessionWindow.this.storeFunctions;
-        }
-
-        @Override public Trigger<M, WindowState<WV>> getTrigger() {
-          return SessionWindow.this.trigger;
-        }
-      };
-    }
-  }
-
-  static <M extends Message, WK, WV, WS extends WindowState<WV>, WM extends WindowOutput<WK, WV>> WindowFn<M, WK, WS, WM> getInternalWindowFn(
-      Window<M, WK, WV, WM> window) {
-    if (window instanceof SessionWindow) {
-      SessionWindow<M, WK, WV> sessionWindow = (SessionWindow<M, WK, WV>) window;
-      return (WindowFn<M, WK, WS, WM>) sessionWindow.getInternalWindowFn();
-    }
-    throw new IllegalArgumentException("Input window type not supported.");
-  }
-
-  /**
-   * Public static API methods start here
-   *
-   */
-
-  /**
-   * The public programming interface class for window function
-   *
-   * @param <M>  the type of input {@link Message}
-   * @param <WK>  the type of key to the {@link Window}
-   * @param <WV>  the type of output value in the {@link WindowOutput}
-   * @param <WM>  the type of message in the window output stream
-   */
-  @InterfaceStability.Unstable
-  public interface Window<M extends Message, WK, WV, WM extends WindowOutput<WK, WV>> {
-
-    /**
-     * Set the triggers for this {@link Window}
-     *
-     * @param wndTrigger  trigger conditions set by the programmers
-     * @return  the {@link Window} function w/ the trigger {@code wndTrigger}
-     */
-    Window<M, WK, WV, WM> setTriggers(TriggerBuilder<M, WV> wndTrigger);
-  }
-
-  /**
-   * Static API method to create a {@link SessionWindow} in which the output value is simply the collection of input messages
-   *
-   * @param sessionKeyFunction  function to calculate session window key
-   * @param <M>  type of input {@link Message}
-   * @param <WK>  type of the session window key
-   * @return  the {@link Window} function for the session
-   */
-  public static <M extends Message, WK> Window<M, WK, Collection<M>, WindowOutput<WK, Collection<M>>> intoSessions(Function<M, WK> sessionKeyFunction) {
-    return new SessionWindow<>(sessionKeyFunction, (m, c) -> {
-        c.add(m);
-        return c;
-      });
-  }
-
-  /**
-   * Static API method to create a {@link SessionWindow} in which the output value is a collection of {@code SI} from the input messages
-   *
-   * @param sessionKeyFunction  function to calculate session window key
-   * @param sessionInfoExtractor  function to retrieve session info of type {@code SI} from the input message of type {@code M}
-   * @param <M>  type of the input {@link Message}
-   * @param <WK>  type of the session window key
-   * @param <SI>  type of the session information retrieved from each input message of type {@code M}
-   * @return  the {@link Window} function for the session
-   */
-  public static <M extends Message, WK, SI> Window<M, WK, Collection<SI>, WindowOutput<WK, Collection<SI>>> intoSessions(Function<M, WK> sessionKeyFunction,
-      Function<M, SI> sessionInfoExtractor) {
-    return new SessionWindow<>(sessionKeyFunction, (m, c) -> {
-        c.add(sessionInfoExtractor.apply(m));
-        return c;
-      });
-  }
-
-  /**
-   * Static API method to create a {@link SessionWindow} as a counter of input messages
-   *
-   * @param sessionKeyFunction  function to calculate session window key
-   * @param <M>  type of the input {@link Message}
-   * @param <WK>  type of the session window key
-   * @return  the {@link Window} function for the session
-   */
-  public static <M extends Message, WK> Window<M, WK, Integer, WindowOutput<WK, Integer>> intoSessionCounter(Function<M, WK> sessionKeyFunction) {
-    return new SessionWindow<>(sessionKeyFunction, (m, c) -> c + 1);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessage.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessage.java b/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessage.java
deleted file mode 100644
index 3c9874d..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessage.java
+++ /dev/null
@@ -1,76 +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.samza.operators.data;
-
-import org.apache.samza.system.IncomingMessageEnvelope;
-import org.apache.samza.system.SystemStreamPartition;
-
-
-/**
- * This class implements a {@link Message} that encapsulates an {@link IncomingMessageEnvelope} from the system
- *
- */
-public class IncomingSystemMessage implements Message<Object, Object>, InputSystemMessage<Offset> {
-  /**
-   * Incoming message envelope
-   */
-  private final IncomingMessageEnvelope imsg;
-
-  /**
-   * The receive time of this incoming message
-   */
-  private final long recvTimeNano;
-
-  /**
-   * Ctor to create a {@code IncomingSystemMessage} from {@link IncomingMessageEnvelope}
-   *
-   * @param imsg The incoming system message
-   */
-  public IncomingSystemMessage(IncomingMessageEnvelope imsg) {
-    this.imsg = imsg;
-    this.recvTimeNano = System.nanoTime();
-  }
-
-  @Override
-  public Object getMessage() {
-    return this.imsg.getMessage();
-  }
-
-  @Override
-  public Object getKey() {
-    return this.imsg.getKey();
-  }
-
-  @Override
-  public long getTimestamp() {
-    return this.recvTimeNano;
-  }
-
-  @Override
-  public Offset getOffset() {
-    // TODO: need to add offset factory to generate different types of offset. This is just a placeholder,
-    // assuming incoming message carries long value as offset (i.e. Kafka case)
-    return new LongOffset(this.imsg.getOffset());
-  }
-
-  @Override
-  public SystemStreamPartition getSystemStreamPartition() {
-    return imsg.getSystemStreamPartition();
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessageEnvelope.java b/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessageEnvelope.java
new file mode 100644
index 0000000..a65809c
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessageEnvelope.java
@@ -0,0 +1,63 @@
+/*
+ * 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.samza.operators.data;
+
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.SystemStreamPartition;
+
+
+/**
+ * A {@link MessageEnvelope} that provides additional information about its input {@link SystemStreamPartition}
+ * and its {@link Offset} within the {@link SystemStreamPartition}.
+ * <p>
+ * Note: the {@link Offset} is only unique and comparable within its {@link SystemStreamPartition}.
+ */
+public class IncomingSystemMessageEnvelope implements MessageEnvelope<Object, Object> {
+
+  private final IncomingMessageEnvelope ime;
+
+  /**
+   * Creates an {@code IncomingSystemMessageEnvelope} from the {@link IncomingMessageEnvelope}.
+   *
+   * @param ime  the {@link IncomingMessageEnvelope} from the input system.
+   */
+  public IncomingSystemMessageEnvelope(IncomingMessageEnvelope ime) {
+    this.ime = ime;
+  }
+
+  @Override
+  public Object getKey() {
+    return this.ime.getKey();
+  }
+
+  @Override
+  public Object getMessage() {
+    return this.ime.getMessage();
+  }
+
+  public Offset getOffset() {
+    // TODO: need to add offset factory to generate different types of offset. This is just a placeholder,
+    // assuming incoming message envelope carries long value as offset (i.e. Kafka case)
+    return new LongOffset(this.ime.getOffset());
+  }
+
+  public SystemStreamPartition getSystemStreamPartition() {
+    return this.ime.getSystemStreamPartition();
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/data/InputSystemMessage.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/data/InputSystemMessage.java b/samza-api/src/main/java/org/apache/samza/operators/data/InputSystemMessage.java
deleted file mode 100644
index 5c23e74..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/data/InputSystemMessage.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.samza.operators.data;
-
-import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.system.SystemStreamPartition;
-
-
-/**
- * This interface defines additional methods a message from an system input should implement, including the methods to
- * get {@link SystemStreamPartition} and the {@link Offset} of the input system message.
- */
-@InterfaceStability.Unstable
-public interface InputSystemMessage<O extends Offset> {
-
-  /**
-   * Get the input message's {@link SystemStreamPartition}
-   *
-   * @return  the {@link SystemStreamPartition} this message is coming from
-   */
-  SystemStreamPartition getSystemStreamPartition();
-
-  /**
-   * Get the offset of the message in the input stream. This should be used to uniquely identify a message in an input stream.
-   *
-   * @return The offset of the message in the input stream.
-   */
-  O getOffset();
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/data/Message.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/data/Message.java b/samza-api/src/main/java/org/apache/samza/operators/data/Message.java
deleted file mode 100644
index 8441682..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/data/Message.java
+++ /dev/null
@@ -1,64 +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.samza.operators.data;
-
-import org.apache.samza.annotation.InterfaceStability;
-
-
-/**
- * This class defines the generic interface of {@link Message}, which is a entry in the input/output stream.
- *
- * <p>The {@link Message} models the basic operatible unit in streaming SQL processes in Samza.
- *
- */
-@InterfaceStability.Unstable
-public interface Message<K, M> {
-
-  /**
-   * Access method to get the corresponding message body in {@link Message}
-   *
-   * @return Message object in this {@link Message}
-   */
-  M getMessage();
-
-  /**
-   * Method to indicate whether this {@link Message} indicates deletion of a message w/ the message key
-   *
-   * @return A boolean value indicates whether the current message is a delete or insert message
-   */
-  default boolean isDelete() {
-    return false;
-  };
-
-  /**
-   * Access method to the key of the message
-   *
-   * @return The key of the message
-   */
-  K getKey();
-
-  /**
-   * Get the message creation timestamp of the message.
-   *
-   * @return The message's timestamp in nano seconds.
-   */
-  long getTimestamp();
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/data/MessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/data/MessageEnvelope.java b/samza-api/src/main/java/org/apache/samza/operators/data/MessageEnvelope.java
new file mode 100644
index 0000000..ad64231
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/data/MessageEnvelope.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.samza.operators.data;
+
+import org.apache.samza.annotation.InterfaceStability;
+
+
+/**
+ * An entry in the input/output {@link org.apache.samza.operators.MessageStream}s.
+ */
+@InterfaceStability.Unstable
+public interface MessageEnvelope<K, M> {
+
+  /**
+   * Get the key for this {@link MessageEnvelope}.
+   *
+   * @return  the key for this {@link MessageEnvelope}
+   */
+  K getKey();
+
+  /**
+   * Get the message in this {@link MessageEnvelope}.
+   *
+   * @return  the message in this {@link MessageEnvelope}
+   */
+  M getMessage();
+
+  /**
+   * Whether this {@link MessageEnvelope} indicates deletion of a previous message with this key.
+   *
+   * @return  true if the current {@link MessageEnvelope} indicates deletion of a previous message with this key
+   */
+  default boolean isDelete() {
+    return false;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/functions/FilterFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/FilterFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/FilterFunction.java
new file mode 100644
index 0000000..e611cd0
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/FilterFunction.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.samza.operators.functions;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.operators.data.MessageEnvelope;
+
+
+/**
+ * A function that specifies whether a {@link MessageEnvelope} should be retained for further processing or filtered out.
+ * @param <M>  type of the input {@link MessageEnvelope}
+ */
+@InterfaceStability.Unstable
+@FunctionalInterface
+public interface FilterFunction<M extends MessageEnvelope> {
+
+  /**
+   * Returns a boolean indicating whether this {@link MessageEnvelope} should be retained or filtered out.
+   * @param message  the {@link MessageEnvelope} to be checked
+   * @return  true if {@link MessageEnvelope} should be retained
+   */
+  boolean apply(M message);
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/functions/FlatMapFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/FlatMapFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/FlatMapFunction.java
new file mode 100644
index 0000000..dbc0bd9
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/FlatMapFunction.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.samza.operators.functions;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.operators.data.MessageEnvelope;
+
+import java.util.Collection;
+
+
+/**
+ * A function that transforms a {@link MessageEnvelope} into a collection of 0 or more {@link MessageEnvelope}s,
+ * possibly of a different type.
+ * @param <M>  type of the input {@link MessageEnvelope}
+ * @param <OM>  type of the transformed {@link MessageEnvelope}s
+ */
+@InterfaceStability.Unstable
+@FunctionalInterface
+public interface FlatMapFunction<M extends MessageEnvelope, OM extends MessageEnvelope> {
+
+  /**
+   * Transforms the provided {@link MessageEnvelope} into a collection of 0 or more {@link MessageEnvelope}s.
+   * @param message  the {@link MessageEnvelope} to be transformed
+   * @return  a collection of 0 or more transformed {@link MessageEnvelope}s
+   */
+  Collection<OM> apply(M message);
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/functions/JoinFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/JoinFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/JoinFunction.java
new file mode 100644
index 0000000..8cb1fce
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/JoinFunction.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.samza.operators.functions;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.operators.data.MessageEnvelope;
+
+
+/**
+ * A function that joins {@link MessageEnvelope}s from two {@link org.apache.samza.operators.MessageStream}s and produces
+ * a joined {@link MessageEnvelope}.
+ * @param <M>  type of the input {@link MessageEnvelope}
+ * @param <JM>  type of the {@link MessageEnvelope} to join with
+ * @param <RM>  type of the joined {@link MessageEnvelope}
+ */
+@InterfaceStability.Unstable
+@FunctionalInterface
+public interface JoinFunction<M extends MessageEnvelope, JM extends MessageEnvelope, RM extends MessageEnvelope> {
+
+  /**
+   * Join the provided {@link MessageEnvelope}s and produces the joined {@link MessageEnvelope}.
+   * @param message  the input {@link MessageEnvelope}
+   * @param otherMessage  the {@link MessageEnvelope} to join with
+   * @return  the joined {@link MessageEnvelope}
+   */
+  RM apply(M message, JM otherMessage);
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/functions/MapFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/MapFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/MapFunction.java
new file mode 100644
index 0000000..04919a7
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/MapFunction.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.samza.operators.functions;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.operators.data.MessageEnvelope;
+
+
+/**
+ * A function that transforms a {@link MessageEnvelope} into another {@link MessageEnvelope}, possibly of a different type.
+ * @param <M>  type of the input {@link MessageEnvelope}
+ * @param <OM>  type of the transformed {@link MessageEnvelope}
+ */
+@InterfaceStability.Unstable
+@FunctionalInterface
+public interface MapFunction<M extends MessageEnvelope, OM extends MessageEnvelope> {
+
+  /**
+   * Transforms the provided {@link MessageEnvelope} into another {@link MessageEnvelope}
+   * @param message  the {@link MessageEnvelope} to be transformed
+   * @return  the transformed {@link MessageEnvelope}
+   */
+  OM apply(M message);
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/functions/SinkFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/SinkFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/SinkFunction.java
new file mode 100644
index 0000000..505da92
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/SinkFunction.java
@@ -0,0 +1,46 @@
+/*
+ * 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.samza.operators.functions;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+
+
+/**
+ * A function that allows sending a {@link MessageEnvelope} to an output system.
+ * @param <M>  type of the input {@link MessageEnvelope}
+ */
+@InterfaceStability.Unstable
+@FunctionalInterface
+public interface SinkFunction<M extends MessageEnvelope> {
+
+  /**
+   * Allows sending the provided {@link MessageEnvelope} to an output {@link org.apache.samza.system.SystemStream} using
+   * the provided {@link MessageCollector}. Also provides access to the {@link TaskCoordinator} to request commits
+   * or shut the container down.
+   *
+   * @param message  the {@link MessageEnvelope} to be sent to an output {@link org.apache.samza.system.SystemStream}
+   * @param messageCollector  the {@link MessageCollector} to use to send the {@link MessageEnvelope}
+   * @param taskCoordinator  the {@link TaskCoordinator} to request commits or shutdown
+   */
+  void apply(M message, MessageCollector messageCollector, TaskCoordinator taskCoordinator);
+
+}


[3/6] samza git commit: SAMZA-1054: Refactor Operator APIs

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
index efa6a96..c77914e 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
@@ -18,88 +18,60 @@
  */
 package org.apache.samza.operators.impl;
 
-import org.apache.samza.operators.data.Message;
 import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.task.MessageCollector;
 import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
-import org.reactivestreams.Processor;
-import org.reactivestreams.Subscriber;
-import org.reactivestreams.Subscription;
 
 import java.util.HashSet;
 import java.util.Set;
 
 
 /**
- * Abstract base class for all stream operator implementation classes.
+ * Abstract base class for all stream operator implementations.
  */
-public abstract class OperatorImpl<M extends Message, RM extends Message>
-    implements Processor<ProcessorContext<M>, ProcessorContext<RM>> {
+public abstract class OperatorImpl<M extends MessageEnvelope, RM extends MessageEnvelope> {
 
-  private final Set<Subscriber<? super ProcessorContext<RM>>> subscribers = new HashSet<>();
-
-  @Override public void subscribe(Subscriber<? super ProcessorContext<RM>> s) {
-    // Only add once
-    subscribers.add(s);
-  }
-
-  @Override public void onSubscribe(Subscription s) {
-
-  }
-
-  @Override public void onNext(ProcessorContext<M> o) {
-
-    onNext(o.getMessage(), o.getCollector(), o.getCoordinator());
-  }
-
-  @Override public void onError(Throwable t) {
-
-  }
-
-  @Override public void onComplete() {
+  private final Set<OperatorImpl<RM, ? extends MessageEnvelope>> nextOperators = new HashSet<>();
 
+  /**
+   * Register the next operator in the chain that this operator should propagate its output to.
+   * @param nextOperator  the next operator in the chain.
+   */
+  void registerNextOperator(OperatorImpl<RM, ? extends MessageEnvelope> nextOperator) {
+    nextOperators.add(nextOperator);
   }
 
   /**
-   * Default method for timer event
+   * Initialize the initial state for stateful operators.
    *
-   * @param nanoTime  the system nano-second when the timer event is triggered
-   * @param collector  the {@link MessageCollector} in the context
-   * @param coordinator  the {@link TaskCoordinator} in the context
+   * @param source  the source that this {@link OperatorImpl} operator is registered with
+   * @param context  the task context to initialize the operator implementation
    */
-  public void onTimer(long nanoTime, MessageCollector collector, TaskCoordinator coordinator) {
-    this.subscribers.forEach(sub -> ((OperatorImpl) sub).onTimer(nanoTime, collector, coordinator));
-  }
+  public void init(MessageStream<M> source, TaskContext context) {}
 
   /**
-   * Each sub-class will implement this method to actually perform the transformation and call the downstream subscribers.
+   * Perform the transformation required for this operator and call the downstream operators.
+   *
+   * Must call {@link #propagateResult} to propage the output to registered downstream operators correctly.
    *
-   * @param message  the input {@link Message}
+   * @param message  the input {@link MessageEnvelope}
    * @param collector  the {@link MessageCollector} in the context
    * @param coordinator  the {@link TaskCoordinator} in the context
    */
-  protected abstract void onNext(M message, MessageCollector collector, TaskCoordinator coordinator);
+  public abstract void onNext(M message, MessageCollector collector, TaskCoordinator coordinator);
 
   /**
-   * Stateful operators will need to override this method to initialize the operators
+   * Helper method to propagate the output of this operator to all registered downstream operators.
    *
-   * @param source  the source that this {@link OperatorImpl} object subscribe to
-   * @param context  the task context to initialize the operators within
-   */
-  protected void init(MessageStream<M> source, TaskContext context) {};
-
-  /**
-   * Method to trigger all downstream operators that consumes the output {@link MessageStream}
-   * from this operator
+   * This method <b>must</b> be called from {@link #onNext} to propagate the operator output correctly.
    *
-   * @param omsg  output {@link Message}
+   * @param outputMessage  output {@link MessageEnvelope}
    * @param collector  the {@link MessageCollector} in the context
    * @param coordinator  the {@link TaskCoordinator} in the context
    */
-  protected void nextProcessors(RM omsg, MessageCollector collector, TaskCoordinator coordinator) {
-    subscribers.forEach(sub ->
-      sub.onNext(new ProcessorContext<>(omsg, collector, coordinator))
-    );
+  void propagateResult(RM outputMessage, MessageCollector collector, TaskCoordinator coordinator) {
+    nextOperators.forEach(sub -> sub.onNext(outputMessage, collector, coordinator));
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java
new file mode 100644
index 0000000..79446be
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java
@@ -0,0 +1,125 @@
+/*
+ * 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.samza.operators.impl;
+
+
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.spec.OperatorSpec;
+import org.apache.samza.operators.spec.PartialJoinOperatorSpec;
+import org.apache.samza.operators.spec.SinkOperatorSpec;
+import org.apache.samza.operators.spec.StreamOperatorSpec;
+import org.apache.samza.operators.spec.WindowOperatorSpec;
+import org.apache.samza.operators.windows.WindowOutput;
+import org.apache.samza.operators.windows.WindowState;
+import org.apache.samza.task.TaskContext;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+
+/**
+ * Instantiates the DAG of {@link OperatorImpl}s corresponding to the {@link OperatorSpec}s for a
+ * {@link MessageStreamImpl}
+ */
+public class OperatorImpls {
+
+  /**
+   * Holds the mapping between the {@link OperatorSpec} and {@link OperatorImpl}s instances.
+   */
+  private static final Map<OperatorSpec, OperatorImpl> OPERATOR_IMPLS = new ConcurrentHashMap<>();
+
+  /**
+   * Traverses the DAG of {@link OperatorSpec}s starting from the provided {@link MessageStreamImpl},
+   * creates the corresponding DAG of {@link OperatorImpl}s, and returns its root {@link RootOperatorImpl} node.
+   *
+   * @param source  the input {@link MessageStreamImpl} to instantiate {@link OperatorImpl}s for
+   * @param <M>  the type of {@link MessageEnvelope}s in the {@code source} {@link MessageStream}
+   * @param context  the {@link TaskContext} required to instantiate operators
+   * @return  root node for the {@link OperatorImpl} DAG
+   */
+  public static <M extends MessageEnvelope> RootOperatorImpl createOperatorImpls(MessageStreamImpl<M> source, TaskContext context) {
+    // since the source message stream might have multiple operator specs registered on it,
+    // create a new root node as a single point of entry for the DAG.
+    RootOperatorImpl<M> rootOperator = new RootOperatorImpl<>();
+    // create the pipeline/topology starting from the source
+    source.getRegisteredOperatorSpecs().forEach(registeredOperator -> {
+        // pass in the source and context s.t. stateful stream operators can initialize their stores
+        OperatorImpl<M, ? extends MessageEnvelope> operatorImpl =
+            createAndRegisterOperatorImpl(registeredOperator, source, context);
+        rootOperator.registerNextOperator(operatorImpl);
+      });
+    return rootOperator;
+  }
+
+  /**
+   * Helper method to recursively traverse the {@link OperatorSpec} DAG and instantiate and link the corresponding
+   * {@link OperatorImpl}s.
+   *
+   * @param operatorSpec  the operatorSpec registered with the {@code source}
+   * @param source  the source {@link MessageStreamImpl}
+   * @param context  the context of the task
+   * @return  the operator implementation for the operatorSpec
+   */
+  private static <M extends MessageEnvelope> OperatorImpl<M, ? extends MessageEnvelope> createAndRegisterOperatorImpl(OperatorSpec operatorSpec,
+      MessageStream source, TaskContext context) {
+    if (!OPERATOR_IMPLS.containsKey(operatorSpec)) {
+      OperatorImpl<M, ? extends MessageEnvelope> operatorImpl = createOperatorImpl(operatorSpec);
+      if (OPERATOR_IMPLS.putIfAbsent(operatorSpec, operatorImpl) == null) {
+        // this is the first time we've added the operatorImpl corresponding to the operatorSpec,
+        // so traverse and initialize and register the rest of the DAG.
+        MessageStream<? extends MessageEnvelope> outStream = operatorSpec.getOutputStream();
+        Collection<OperatorSpec> registeredSpecs = ((MessageStreamImpl) outStream).getRegisteredOperatorSpecs();
+        registeredSpecs.forEach(registeredSpec -> {
+            OperatorImpl subImpl = createAndRegisterOperatorImpl(registeredSpec, outStream, context);
+            operatorImpl.registerNextOperator(subImpl);
+          });
+        operatorImpl.init(source, context);
+        return operatorImpl;
+      }
+    }
+
+    // the implementation corresponding to operatorSpec has already been instantiated
+    // and registered, so we do not need to traverse the DAG further.
+    return OPERATOR_IMPLS.get(operatorSpec);
+  }
+
+  /**
+   * Creates a new {@link OperatorImpl} instance for the provided {@link OperatorSpec}.
+   *
+   * @param operatorSpec  the immutable {@link OperatorSpec} definition.
+   * @param <M>  type of input {@link MessageEnvelope}
+   * @return  the {@link OperatorImpl} implementation instance
+   */
+  protected static <M extends MessageEnvelope> OperatorImpl<M, ? extends MessageEnvelope> createOperatorImpl(OperatorSpec operatorSpec) {
+    if (operatorSpec instanceof StreamOperatorSpec) {
+      return new StreamOperatorImpl<>((StreamOperatorSpec<M, ? extends MessageEnvelope>) operatorSpec);
+    } else if (operatorSpec instanceof SinkOperatorSpec) {
+      return new SinkOperatorImpl<>((SinkOperatorSpec<M>) operatorSpec);
+    } else if (operatorSpec instanceof WindowOperatorSpec) {
+      return new SessionWindowOperatorImpl<>((WindowOperatorSpec<M, ?, ? extends WindowState, ? extends WindowOutput>) operatorSpec);
+    } else if (operatorSpec instanceof PartialJoinOperatorSpec) {
+      return new PartialJoinOperatorImpl<>((PartialJoinOperatorSpec) operatorSpec);
+    }
+    throw new IllegalArgumentException(
+        String.format("Unsupported OperatorSpec: %s", operatorSpec.getClass().getName()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java
new file mode 100644
index 0000000..90569b4
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java
@@ -0,0 +1,46 @@
+/*
+ * 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.samza.operators.impl;
+
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.spec.PartialJoinOperatorSpec;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+
+
+/**
+ * Implementation of a {@link PartialJoinOperatorSpec}. This class implements function
+ * that only takes in one input stream among all inputs to the join and generate the join output.
+ *
+ * @param <M>  type of {@link MessageEnvelope}s in the input stream
+ * @param <JM>  type of {@link MessageEnvelope}s in the stream to join with
+ * @param <RM>  type of {@link MessageEnvelope}s in the joined stream
+ */
+class PartialJoinOperatorImpl<M extends MessageEnvelope<K, ?>, K, JM extends MessageEnvelope<K, ?>, RM extends MessageEnvelope>
+    extends OperatorImpl<M, RM> {
+
+  PartialJoinOperatorImpl(PartialJoinOperatorSpec<M, K, JM, RM> joinOp) {
+    // TODO: implement PartialJoinOperatorImpl constructor
+  }
+
+  @Override
+  public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
+    // TODO: implement PartialJoinOperatorImpl processing logic
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
deleted file mode 100644
index cc7ef2b..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
+++ /dev/null
@@ -1,53 +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.samza.operators.impl;
-
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-
-
-/**
- * Wrapper class to be used by {@link OperatorImpl}
- *
- * @param <M>  Type of input stream {@link Message}
- */
-public class ProcessorContext<M extends Message> {
-  private final M message;
-  private final MessageCollector collector;
-  private final TaskCoordinator coordinator;
-
-  ProcessorContext(M message, MessageCollector collector, TaskCoordinator coordinator) {
-    this.message = message;
-    this.collector = collector;
-    this.coordinator = coordinator;
-  }
-
-  M getMessage() {
-    return this.message;
-  }
-
-  MessageCollector getCollector() {
-    return this.collector;
-  }
-
-  TaskCoordinator getCoordinator() {
-    return this.coordinator;
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.java
new file mode 100644
index 0000000..7132b86
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.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.samza.operators.impl;
+
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+
+
+/**
+ * A no-op operator implementation that forwards incoming {@link MessageEnvelope}s to all of its subscribers.
+ * @param <M>  type of incoming {@link MessageEnvelope}s
+ */
+final class RootOperatorImpl<M extends MessageEnvelope> extends OperatorImpl<M, M> {
+
+  @Override
+  public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
+    this.propagateResult(message, collector, coordinator);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java
new file mode 100644
index 0000000..e8a635c
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.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.samza.operators.impl;
+
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.StateStoreImpl;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.spec.WindowOperatorSpec;
+import org.apache.samza.operators.windows.WindowState;
+import org.apache.samza.operators.windows.WindowOutput;
+import org.apache.samza.storage.kv.Entry;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+
+
+/**
+ * Default implementation class of a {@link WindowOperatorSpec} for a session window.
+ *
+ * @param <M>  the type of input {@link MessageEnvelope}
+ * @param <RK>  the type of window key
+ * @param <WS>  the type of window state
+ * @param <RM>  the type of aggregated value of the window
+ */
+class SessionWindowOperatorImpl<M extends MessageEnvelope, RK, WS extends WindowState, RM extends WindowOutput<RK, ?>>
+    extends OperatorImpl<M, RM> {
+
+  private final WindowOperatorSpec<M, RK, WS, RM> windowSpec;
+  private StateStoreImpl<M, RK, WS> stateStore = null;
+
+  SessionWindowOperatorImpl(WindowOperatorSpec<M, RK, WS, RM> windowSpec) {
+    this.windowSpec = windowSpec;
+  }
+
+  @Override
+  public void init(MessageStream<M> source, TaskContext context) {
+    this.stateStore = new StateStoreImpl<>(this.windowSpec.getStoreFns(), windowSpec.getStoreName(source));
+    this.stateStore.init(context);
+  }
+
+  @Override
+  public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
+    Entry<RK, WS> state = this.stateStore.getState(message);
+    this.propagateResult(this.windowSpec.getTransformFn().apply(message, state), collector, coordinator);
+    this.stateStore.updateState(message, state);
+  }
+
+  public void onTimer(MessageCollector collector, TaskCoordinator coordinator) {
+    // This is to periodically check the timeout triggers to get the list of window states to be updated
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
deleted file mode 100644
index b0f4f27..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.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.samza.operators.impl;
-
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.internal.Operators.StreamOperator;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-
-import java.util.Collection;
-import java.util.function.Function;
-
-
-/**
- * Base class for all implementation of operators
- *
- * @param <M>  type of message in the input stream
- * @param <RM>  type of message in the output stream
- */
-public class SimpleOperatorImpl<M extends Message, RM extends Message> extends OperatorImpl<M, RM> {
-
-  private final Function<M, Collection<RM>> transformFn;
-
-  SimpleOperatorImpl(StreamOperator<M, RM> op) {
-    super();
-    this.transformFn = op.getFunction();
-  }
-
-  @Override protected void onNext(M imsg, MessageCollector collector, TaskCoordinator coordinator) {
-    // actually calling the transform function and then for each output, call nextProcessors()
-    this.transformFn.apply(imsg).forEach(r -> this.nextProcessors(r, collector, coordinator));
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
index a8a639e..abed03f 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
@@ -18,24 +18,26 @@
  */
 package org.apache.samza.operators.impl;
 
-import org.apache.samza.operators.internal.Operators.SinkOperator;
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.data.Message;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.operators.spec.SinkOperatorSpec;
 import org.apache.samza.task.MessageCollector;
 import org.apache.samza.task.TaskCoordinator;
 
 
 /**
- * Implementation for {@link SinkOperator}
+ * Implementation for {@link SinkOperatorSpec}
  */
-public class SinkOperatorImpl<M extends Message> extends OperatorImpl<M, Message> {
-  private final MessageStream.VoidFunction3<M, MessageCollector, TaskCoordinator> sinkFunc;
+class SinkOperatorImpl<M extends MessageEnvelope> extends OperatorImpl<M, MessageEnvelope> {
 
-  SinkOperatorImpl(SinkOperator<M> sinkOp) {
-    this.sinkFunc = sinkOp.getFunction();
+  private final SinkFunction<M> sinkFn;
+
+  SinkOperatorImpl(SinkOperatorSpec<M> sinkOp) {
+    this.sinkFn = sinkOp.getSinkFn();
   }
 
-  @Override protected void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
-    this.sinkFunc.apply(message, collector, coordinator);
+  @Override
+  public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
+    this.sinkFn.apply(message, collector, coordinator);
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
deleted file mode 100644
index 7840b5b..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
+++ /dev/null
@@ -1,56 +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.samza.operators.impl;
-
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.internal.Operators.StoreFunctions;
-import org.apache.samza.storage.kv.Entry;
-import org.apache.samza.storage.kv.KeyValueStore;
-import org.apache.samza.task.TaskContext;
-
-
-/**
- * The base class for all state stores
- */
-public class StateStoreImpl<M extends Message, SK, SS> {
-  private final String storeName;
-  private final StoreFunctions<M, SK, SS> storeFunctions;
-  private KeyValueStore<SK, SS> kvStore = null;
-
-  public StateStoreImpl(StoreFunctions<M, SK, SS> store, String storeName) {
-    this.storeFunctions = store;
-    this.storeName = storeName;
-  }
-
-  public void init(TaskContext context) {
-    this.kvStore = (KeyValueStore<SK, SS>) context.getStore(this.storeName);
-  }
-
-  public Entry<SK, SS> getState(M m) {
-    SK key = this.storeFunctions.getStoreKeyFinder().apply(m);
-    SS state = this.kvStore.get(key);
-    return new Entry<>(key, state);
-  }
-
-  public Entry<SK, SS> updateState(M m, Entry<SK, SS> oldEntry) {
-    SS newValue = this.storeFunctions.getStateUpdater().apply(m, oldEntry.getValue());
-    this.kvStore.put(oldEntry.getKey(), newValue);
-    return new Entry<>(oldEntry.getKey(), newValue);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java
new file mode 100644
index 0000000..3a5c56e
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.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.samza.operators.impl;
+
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.spec.StreamOperatorSpec;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+
+
+/**
+ * A StreamOperator that accepts a 1:n transform function and applies it to each incoming {@link MessageEnvelope}.
+ *
+ * @param <M>  type of {@link MessageEnvelope} in the input stream
+ * @param <RM>  type of {@link MessageEnvelope} in the output stream
+ */
+class StreamOperatorImpl<M extends MessageEnvelope, RM extends MessageEnvelope> extends OperatorImpl<M, RM> {
+
+  private final FlatMapFunction<M, RM> transformFn;
+
+  StreamOperatorImpl(StreamOperatorSpec<M, RM> streamOperatorSpec) {
+    this.transformFn = streamOperatorSpec.getTransformFn();
+  }
+
+  @Override
+  public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
+    // call the transform function and then for each output call propagateResult()
+    this.transformFn.apply(message).forEach(r -> this.propagateResult(r, collector, coordinator));
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/join/PartialJoinOpImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/join/PartialJoinOpImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/join/PartialJoinOpImpl.java
deleted file mode 100644
index 4238d45..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/join/PartialJoinOpImpl.java
+++ /dev/null
@@ -1,44 +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.samza.operators.impl.join;
-
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.internal.Operators.PartialJoinOperator;
-import org.apache.samza.operators.impl.OperatorImpl;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-
-
-/**
- * Implementation of a {@link PartialJoinOperator}. This class implements function
- * that only takes in one input stream among all inputs to the join and generate the join output.
- *
- * @param <M>  Type of input stream {@link Message}
- * @param <RM>  Type of join output stream {@link Message}
- */
-public class PartialJoinOpImpl<M extends Message<K, ?>, K, JM extends Message<K, ?>, RM extends Message> extends OperatorImpl<M, RM> {
-
-  public PartialJoinOpImpl(PartialJoinOperator<M, K, JM, RM> joinOp) {
-    // TODO: implement PartialJoinOpImpl constructor
-  }
-
-  @Override protected void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
-    // TODO: implement PartialJoinOpImpl processing logic
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
deleted file mode 100644
index 0d6141e..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
+++ /dev/null
@@ -1,65 +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.samza.operators.impl.window;
-
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.WindowState;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.internal.Operators.WindowOperator;
-import org.apache.samza.operators.internal.WindowOutput;
-import org.apache.samza.operators.impl.OperatorImpl;
-import org.apache.samza.operators.impl.StateStoreImpl;
-import org.apache.samza.storage.kv.Entry;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskContext;
-import org.apache.samza.task.TaskCoordinator;
-
-
-/**
- * Default implementation class of a {@link WindowOperator} for a session window.
- *
- * @param <M>  the type of input {@link Message}
- * @param <RK>  the type of window key
- * @param <WS>  the type of window state
- * @param <RM>  the type of aggregated value of the window
- */
-public class SessionWindowImpl<M extends Message, RK, WS extends WindowState, RM extends WindowOutput<RK, ?>> extends
-    OperatorImpl<M, RM> {
-  private final WindowOperator<M, RK, WS, RM> sessWnd;
-  private StateStoreImpl<M, RK, WS> wndStore = null;
-
-  public SessionWindowImpl(WindowOperator<M, RK, WS, RM> sessWnd) {
-    this.sessWnd = sessWnd;
-  }
-
-  @Override protected void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
-    Entry<RK, WS> state = this.wndStore.getState(message);
-    this.nextProcessors(this.sessWnd.getFunction().apply(message, state), collector, coordinator);
-    this.wndStore.updateState(message, state);
-  }
-
-  public void onTimer(MessageCollector collector, TaskCoordinator coordinator) {
-    // This is to periodically check the timeout triggers to get the list of window states to be updated
-  }
-
-  @Override protected void init(MessageStream<M> source, TaskContext context) {
-    this.wndStore = new StateStoreImpl<>(this.sessWnd.getStoreFunctions(), sessWnd.getStoreName(source));
-    this.wndStore.init(context);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
new file mode 100644
index 0000000..8b75cdc
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpec.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.samza.operators.spec;
+
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.data.MessageEnvelope;
+
+
+/**
+ * A stateless serializable stream operator specification that holds all the information required
+ * to transform the input {@link MessageStream} and produce the output {@link MessageStream}.
+ */
+public interface OperatorSpec<OM extends MessageEnvelope> {
+
+  /**
+   * Get the output stream containing transformed {@link MessageEnvelope} produced by this operator.
+   * @return  the output stream containing transformed {@link MessageEnvelope} produced by this operator.
+   */
+  MessageStream<OM> getOutputStream();
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java
new file mode 100644
index 0000000..f622b34
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.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.samza.operators.spec;
+
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.windows.WindowState;
+import org.apache.samza.operators.windows.WindowFn;
+import org.apache.samza.operators.windows.WindowOutput;
+
+import java.util.ArrayList;
+import java.util.UUID;
+import java.util.function.BiFunction;
+
+
+/**
+ * Factory methods for creating {@link OperatorSpec} instances.
+ */
+public class OperatorSpecs {
+
+  private OperatorSpecs() {}
+
+  private static String getOperatorId() {
+    // TODO: need to change the IDs to be a consistent, durable IDs that can be recovered across container and job restarts
+    return UUID.randomUUID().toString();
+  }
+
+  /**
+   * Creates a {@link StreamOperatorSpec}.
+   *
+   * @param transformFn  the transformation function
+   * @param <M>  type of input {@link MessageEnvelope}
+   * @param <OM>  type of output {@link MessageEnvelope}
+   * @return  the {@link StreamOperatorSpec}
+   */
+  public static <M extends MessageEnvelope, OM extends MessageEnvelope> StreamOperatorSpec<M, OM> createStreamOperator(
+      FlatMapFunction<M, OM> transformFn) {
+    return new StreamOperatorSpec<>(transformFn);
+  }
+
+  /**
+   * Creates a {@link SinkOperatorSpec}.
+   *
+   * @param sinkFn  the sink function
+   * @param <M>  type of input {@link MessageEnvelope}
+   * @return  the {@link SinkOperatorSpec}
+   */
+  public static <M extends MessageEnvelope> SinkOperatorSpec<M> createSinkOperator(SinkFunction<M> sinkFn) {
+    return new SinkOperatorSpec<>(sinkFn);
+  }
+
+  /**
+   * Creates a {@link WindowOperatorSpec}.
+   *
+   * @param windowFn  the {@link WindowFn} function
+   * @param <M>  type of input {@link MessageEnvelope}
+   * @param <WK>  type of window key
+   * @param <WS>  type of {@link WindowState}
+   * @param <WM>  type of output {@link WindowOutput}
+   * @return  the {@link WindowOperatorSpec}
+   */
+  public static <M extends MessageEnvelope, WK, WS extends WindowState, WM extends WindowOutput<WK, ?>> WindowOperatorSpec<M, WK, WS, WM> createWindowOperator(
+      WindowFn<M, WK, WS, WM> windowFn) {
+    return new WindowOperatorSpec<>(windowFn, OperatorSpecs.getOperatorId());
+  }
+
+  /**
+   * Creates a {@link PartialJoinOperatorSpec}.
+   *
+   * @param partialJoinFn  the join function
+   * @param joinOutput  the output {@link MessageStreamImpl}
+   * @param <M>  type of input {@link MessageEnvelope}
+   * @param <K>  type of join key
+   * @param <JM>  the type of {@link MessageEnvelope} in the other join stream
+   * @param <OM>  the type of {@link MessageEnvelope} in the join output
+   * @return  the {@link PartialJoinOperatorSpec}
+   */
+  public static <M extends MessageEnvelope<K, ?>, K, JM extends MessageEnvelope<K, ?>, OM extends MessageEnvelope> PartialJoinOperatorSpec<M, K, JM, OM> createPartialJoinOperator(
+      BiFunction<M, JM, OM> partialJoinFn, MessageStreamImpl<OM> joinOutput) {
+    return new PartialJoinOperatorSpec<>(partialJoinFn, joinOutput, OperatorSpecs.getOperatorId());
+  }
+
+  /**
+   * Creates a {@link StreamOperatorSpec} with a merger function.
+   *
+   * @param mergeOutput  the output {@link MessageStreamImpl} from the merger
+   * @param <M>  the type of input {@link MessageEnvelope}
+   * @return  the {@link StreamOperatorSpec} for the merge
+   */
+  public static <M extends MessageEnvelope> StreamOperatorSpec<M, M> createMergeOperator(MessageStreamImpl<M> mergeOutput) {
+    return new StreamOperatorSpec<M, M>(t ->
+      new ArrayList<M>() { {
+          this.add(t);
+        } },
+      mergeOutput);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java
new file mode 100644
index 0000000..f74f35d
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java
@@ -0,0 +1,104 @@
+/*
+ * 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.samza.operators.spec;
+
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.windows.StoreFunctions;
+
+import java.util.function.BiFunction;
+
+
+/**
+ * Spec for the partial join operator that takes {@link MessageEnvelope}s from one input stream, joins with buffered
+ * {@link MessageEnvelope}s from another stream, and produces join results to an output {@link MessageStreamImpl}.
+ *
+ * @param <M>  the type of input {@link MessageEnvelope}
+ * @param <K>  the type of join key
+ * @param <JM>  the type of {@link MessageEnvelope} in the other join stream
+ * @param <RM>  the type of {@link MessageEnvelope} in the join output stream
+ */
+public class PartialJoinOperatorSpec<M extends MessageEnvelope<K, ?>, K, JM extends MessageEnvelope<K, ?>, RM extends MessageEnvelope>
+    implements OperatorSpec<RM> {
+
+  private final MessageStreamImpl<RM> joinOutput;
+
+  /**
+   * The transformation function of {@link PartialJoinOperatorSpec} that takes an input {@link MessageEnvelope} of
+   * type {@code M}, joins with a stream of buffered {@link MessageEnvelope}s of type {@code JM} from another stream,
+   * and generates a joined result {@link MessageEnvelope} of type {@code RM}.
+   */
+  private final BiFunction<M, JM, RM> transformFn;
+
+  /**
+   * The {@link MessageEnvelope} store functions that read the buffered {@link MessageEnvelope}s from the other
+   * stream in the join.
+   */
+  private final StoreFunctions<JM, K, JM> joinStoreFns;
+
+  /**
+   * The {@link MessageEnvelope} store functions that save the buffered {@link MessageEnvelope} of this
+   * {@link MessageStreamImpl} in the join.
+   */
+  private final StoreFunctions<M, K, M> selfStoreFns;
+
+  /**
+   * The unique ID for this operator.
+   */
+  private final String operatorId;
+
+  /**
+   * Default constructor for a {@link PartialJoinOperatorSpec}.
+   *
+   * @param partialJoinFn  partial join function that take type {@code M} of input {@link MessageEnvelope} and join
+   *                       w/ type {@code JM} of buffered {@link MessageEnvelope} from another stream
+   * @param joinOutput  the output {@link MessageStreamImpl} of the join results
+   */
+  PartialJoinOperatorSpec(BiFunction<M, JM, RM> partialJoinFn, MessageStreamImpl<RM> joinOutput, String operatorId) {
+    this.joinOutput = joinOutput;
+    this.transformFn = partialJoinFn;
+    // Read-only join store, no creator/updater functions required.
+    this.joinStoreFns = new StoreFunctions<>(m -> m.getKey(), null);
+    // Buffered message envelope store for this input stream.
+    this.selfStoreFns = new StoreFunctions<>(m -> m.getKey(), (m, s1) -> m);
+    this.operatorId = operatorId;
+  }
+
+  @Override
+  public String toString() {
+    return this.operatorId;
+  }
+
+  @Override
+  public MessageStreamImpl<RM> getOutputStream() {
+    return this.joinOutput;
+  }
+
+  public StoreFunctions<JM, K, JM> getJoinStoreFns() {
+    return this.joinStoreFns;
+  }
+
+  public StoreFunctions<M, K, M> getSelfStoreFns() {
+    return this.selfStoreFns;
+  }
+
+  public BiFunction<M, JM, RM> getTransformFn() {
+    return this.transformFn;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java
new file mode 100644
index 0000000..4348bc0
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.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.samza.operators.spec;
+
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.operators.MessageStreamImpl;
+
+
+/**
+ * The spec for a sink operator that accepts user-defined logic to output a {@link MessageStreamImpl} to an external
+ * system. This is a terminal operator and does allows further operator chaining.
+ *
+ * @param <M>  the type of input {@link MessageEnvelope}
+ */
+public class SinkOperatorSpec<M extends MessageEnvelope> implements OperatorSpec {
+
+  /**
+   * The user-defined sink function
+   */
+  private final SinkFunction<M> sinkFn;
+
+  /**
+   * Default constructor for a {@link SinkOperatorSpec}.
+   *
+   * @param sinkFn  a user defined {@link SinkFunction} that will be called with the output {@link MessageEnvelope},
+   *                the output {@link org.apache.samza.task.MessageCollector} and the
+   *                {@link org.apache.samza.task.TaskCoordinator}.
+   */
+  SinkOperatorSpec(SinkFunction<M> sinkFn) {
+    this.sinkFn = sinkFn;
+  }
+
+  /**
+   * This is a terminal operator and doesn't allow further operator chaining.
+   * @return  null
+   */
+  @Override
+  public MessageStreamImpl getOutputStream() {
+    return null;
+  }
+
+  public SinkFunction<M> getSinkFn() {
+    return this.sinkFn;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java
new file mode 100644
index 0000000..ed18da4
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.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.samza.operators.spec;
+
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.MessageStreamImpl;
+
+
+/**
+ * The spec for a linear stream operator that outputs 0 or more {@link MessageEnvelope}s for each input {@link MessageEnvelope}.
+ *
+ * @param <M>  the type of input {@link MessageEnvelope}
+ * @param <OM>  the type of output {@link MessageEnvelope}
+ */
+public class StreamOperatorSpec<M extends MessageEnvelope, OM extends MessageEnvelope> implements OperatorSpec<OM> {
+
+  private final MessageStreamImpl<OM> outputStream;
+
+  private final FlatMapFunction<M, OM> transformFn;
+
+  /**
+   * Default constructor for a {@link StreamOperatorSpec}.
+   *
+   * @param transformFn  the transformation function that transforms each input {@link MessageEnvelope} into a collection
+   *                     of output {@link MessageEnvelope}s
+   */
+  StreamOperatorSpec(FlatMapFunction<M, OM> transformFn) {
+    this(transformFn, new MessageStreamImpl<>());
+  }
+
+  /**
+   * Constructor for a {@link StreamOperatorSpec} that accepts an output {@link MessageStreamImpl}.
+   *
+   * @param transformFn  the transformation function
+   * @param outputStream  the output {@link MessageStreamImpl}
+   */
+  StreamOperatorSpec(FlatMapFunction<M, OM> transformFn, MessageStreamImpl<OM> outputStream) {
+    this.outputStream = outputStream;
+    this.transformFn = transformFn;
+  }
+
+  @Override
+  public MessageStreamImpl<OM> getOutputStream() {
+    return this.outputStream;
+  }
+
+  public FlatMapFunction<M, OM> getTransformFn() {
+    return this.transformFn;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java
new file mode 100644
index 0000000..2f5b1e7
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.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.samza.operators.spec;
+
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.windows.StoreFunctions;
+import org.apache.samza.operators.windows.Trigger;
+import org.apache.samza.operators.windows.WindowFn;
+import org.apache.samza.operators.windows.WindowOutput;
+import org.apache.samza.operators.windows.WindowState;
+import org.apache.samza.storage.kv.Entry;
+
+import java.util.function.BiFunction;
+
+
+/**
+ * Defines a window operator that takes one {@link MessageStreamImpl} as an input, accumulates the window state,
+ * and generates an output {@link MessageStreamImpl} with output type {@code WM} which extends {@link WindowOutput}
+ *
+ * @param <M>  the type of input {@link MessageEnvelope}
+ * @param <WK>  the type of key in the output {@link MessageEnvelope} from the {@link WindowOperatorSpec} function
+ * @param <WS>  the type of window state in the {@link WindowOperatorSpec} function
+ * @param <WM>  the type of window output {@link MessageEnvelope}
+ */
+public class WindowOperatorSpec<M extends MessageEnvelope, WK, WS extends WindowState, WM extends WindowOutput<WK, ?>> implements
+    OperatorSpec<WM> {
+
+  /**
+   * The output {@link MessageStream}.
+   */
+  private final MessageStreamImpl<WM> outputStream;
+
+  /**
+   * The window transformation function that takes {@link MessageEnvelope}s from one input stream, aggregates with the window
+   * state(s) from the window state store, and generate output {@link MessageEnvelope}s for the output stream.
+   */
+  private final BiFunction<M, Entry<WK, WS>, WM> transformFn;
+
+  /**
+   * The state store functions for the {@link WindowOperatorSpec}.
+   */
+  private final StoreFunctions<M, WK, WS> storeFns;
+
+  /**
+   * The window trigger.
+   */
+  private final Trigger<M, WS> trigger;
+
+  /**
+   * The unique ID of this operator.
+   */
+  private final String operatorId;
+
+  /**
+   * Constructor for {@link WindowOperatorSpec}.
+   *
+   * @param windowFn  the window function
+   * @param operatorId  auto-generated unique ID of this operator
+   */
+  WindowOperatorSpec(WindowFn<M, WK, WS, WM> windowFn, String operatorId) {
+    this.outputStream = new MessageStreamImpl<>();
+    this.transformFn = windowFn.getTransformFn();
+    this.storeFns = windowFn.getStoreFns();
+    this.trigger = windowFn.getTrigger();
+    this.operatorId = operatorId;
+  }
+
+  @Override
+  public String toString() {
+    return this.operatorId;
+  }
+
+  @Override
+  public MessageStreamImpl<WM> getOutputStream() {
+    return this.outputStream;
+  }
+
+  public StoreFunctions<M, WK, WS> getStoreFns() {
+    return this.storeFns;
+  }
+
+  public BiFunction<M, Entry<WK, WS>, WM> getTransformFn() {
+    return this.transformFn;
+  }
+
+  public Trigger<M, WS> getTrigger() {
+    return this.trigger;
+  }
+
+  /**
+   * Method to generate the window operator's state store name
+   * TODO HIGH pmaheshw: should this be here?
+   *
+   * @param inputStream the input {@link MessageStreamImpl} to this state store
+   * @return   the persistent store name of the window operator
+   */
+  public String getStoreName(MessageStream<M> inputStream) {
+    //TODO: need to get the persistent name of ds and the operator in a serialized form
+    return String.format("input-%s-wndop-%s", inputStream.toString(), this.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java b/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
deleted file mode 100644
index c2f780d..0000000
--- a/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
+++ /dev/null
@@ -1,85 +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.samza.task;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.MessageStreams;
-import org.apache.samza.operators.MessageStreams.SystemMessageStream;
-import org.apache.samza.operators.data.IncomingSystemMessage;
-import org.apache.samza.operators.impl.ChainedOperators;
-import org.apache.samza.operators.task.StreamOperatorTask;
-import org.apache.samza.system.IncomingMessageEnvelope;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.util.HashMap;
-import java.util.Map;
-
-
-/**
- * An adaptor task class that invoke the user-implemented (@link StreamOperatorTask}s via {@link org.apache.samza.operators.MessageStream} programming APIs
- *
- */
-public final class StreamOperatorAdaptorTask implements StreamTask, InitableTask, WindowableTask {
-  /**
-   * A map with entries mapping {@link SystemStreamPartition} to {@link org.apache.samza.operators.impl.ChainedOperators} that takes the {@link SystemStreamPartition}
-   * as the input stream
-   */
-  private final Map<SystemStreamPartition, ChainedOperators> operatorChains = new HashMap<>();
-
-  /**
-   * Wrapped {@link StreamOperatorTask} class
-   */
-  private final StreamOperatorTask userTask;
-
-  /**
-   * Constructor that wraps the user-defined {@link StreamOperatorTask}
-   *
-   * @param userTask  the user-defined {@link StreamOperatorTask}
-   */
-  public StreamOperatorAdaptorTask(StreamOperatorTask userTask) {
-    this.userTask = userTask;
-  }
-
-  @Override
-  public final void init(Config config, TaskContext context) throws Exception {
-    if (this.userTask instanceof InitableTask) {
-      ((InitableTask) this.userTask).init(config, context);
-    }
-    Map<SystemStreamPartition, SystemMessageStream> sources = new HashMap<>();
-    context.getSystemStreamPartitions().forEach(ssp -> {
-        SystemMessageStream ds = MessageStreams.input(ssp);
-        sources.put(ssp, ds);
-      });
-    this.userTask.initOperators(sources.values());
-    sources.forEach((ssp, ds) -> operatorChains.put(ssp, ChainedOperators.create(ds, context)));
-  }
-
-  @Override
-  public final void process(IncomingMessageEnvelope ime, MessageCollector collector, TaskCoordinator coordinator) {
-    this.operatorChains.get(ime.getSystemStreamPartition()).onNext(new IncomingSystemMessage(ime), collector, coordinator);
-  }
-
-  @Override
-  public final void window(MessageCollector collector, TaskCoordinator coordinator) throws Exception {
-    this.operatorChains.forEach((ssp, chain) -> chain.onTimer(collector, coordinator));
-    if (this.userTask instanceof WindowableTask) {
-      ((WindowableTask) this.userTask).window(collector, coordinator);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/BroadcastTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/BroadcastTask.java b/samza-operator/src/test/java/org/apache/samza/operators/BroadcastTask.java
new file mode 100644
index 0000000..e45d068
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/BroadcastTask.java
@@ -0,0 +1,101 @@
+/*
+ * 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.samza.operators;
+
+
+import org.apache.samza.operators.data.IncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.Offset;
+import org.apache.samza.operators.windows.TriggerBuilder;
+import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.util.Collection;
+import java.util.Map;
+
+
+/**
+ * Example implementation of split stream tasks
+ *
+ */
+public class BroadcastTask implements StreamOperatorTask {
+  class MessageType {
+    String field1;
+    String field2;
+    String field3;
+    String field4;
+    String parKey;
+    private long timestamp;
+
+    public long getTimestamp() {
+      return this.timestamp;
+    }
+  }
+
+  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
+    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
+      super(key, data, offset, partition);
+    }
+  }
+
+  @Override
+  public void transform(Map<SystemStreamPartition, MessageStream<IncomingSystemMessageEnvelope>> messageStreams) {
+    messageStreams.values().forEach(entry -> {
+        MessageStream<JsonMessageEnvelope> inputStream = entry.map(this::getInputMessage);
+
+        inputStream.filter(this::myFilter1).
+          window(Windows.<JsonMessageEnvelope, String>intoSessionCounter(
+              m -> String.format("%s-%s", m.getMessage().field1, m.getMessage().field2)).
+            setTriggers(TriggerBuilder.<JsonMessageEnvelope, Integer>earlyTriggerWhenExceedWndLen(100).
+              addLateTriggerOnSizeLimit(10).
+              addTimeoutSinceLastMessage(30000)));
+
+        inputStream.filter(this::myFilter2).
+          window(Windows.<JsonMessageEnvelope, String>intoSessions(
+              m -> String.format("%s-%s", m.getMessage().field3, m.getMessage().field4)).
+            setTriggers(TriggerBuilder.<JsonMessageEnvelope, Collection<JsonMessageEnvelope>>earlyTriggerWhenExceedWndLen(100).
+              addTimeoutSinceLastMessage(30000)));
+
+        inputStream.filter(this::myFilter3).
+          window(Windows.<JsonMessageEnvelope, String, MessageType>intoSessions(
+              m -> String.format("%s-%s", m.getMessage().field3, m.getMessage().field4), m -> m.getMessage()).
+            setTriggers(TriggerBuilder.<JsonMessageEnvelope, Collection<MessageType>>earlyTriggerOnEventTime(m -> m.getMessage().getTimestamp(), 30000).
+              addTimeoutSinceFirstMessage(60000)));
+      });
+  }
+
+  JsonMessageEnvelope getInputMessage(IncomingSystemMessageEnvelope m1) {
+    return (JsonMessageEnvelope) m1.getMessage();
+  }
+
+  boolean myFilter1(JsonMessageEnvelope m1) {
+    // Do user defined processing here
+    return m1.getMessage().parKey.equals("key1");
+  }
+
+  boolean myFilter2(JsonMessageEnvelope m1) {
+    // Do user defined processing here
+    return m1.getMessage().parKey.equals("key2");
+  }
+
+  boolean myFilter3(JsonMessageEnvelope m1) {
+    return m1.getMessage().parKey.equals("key3");
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/JoinTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/JoinTask.java b/samza-operator/src/test/java/org/apache/samza/operators/JoinTask.java
new file mode 100644
index 0000000..1b10609
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/JoinTask.java
@@ -0,0 +1,77 @@
+/*
+ * 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.samza.operators;
+
+import org.apache.samza.operators.data.IncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.Offset;
+import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * Example implementation of unique key-based stream-stream join tasks
+ *
+ */
+public class JoinTask implements StreamOperatorTask {
+  class MessageType {
+    String joinKey;
+    List<String> joinFields = new ArrayList<>();
+  }
+
+  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
+    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
+      super(key, data, offset, partition);
+    }
+  }
+
+  MessageStream<JsonMessageEnvelope> joinOutput = null;
+
+  @Override
+  public void transform(Map<SystemStreamPartition, MessageStream<IncomingSystemMessageEnvelope>> messageStreams) {
+    messageStreams.values().forEach(messageStream -> {
+        MessageStream<JsonMessageEnvelope> newSource = messageStream.map(this::getInputMessage);
+        if (joinOutput == null) {
+          joinOutput = newSource;
+        } else {
+          joinOutput = joinOutput.join(newSource, (m1, m2) -> this.myJoinResult(m1, m2));
+        }
+      });
+  }
+
+  private JsonMessageEnvelope getInputMessage(IncomingSystemMessageEnvelope ism) {
+    return new JsonMessageEnvelope(
+        ((MessageType) ism.getMessage()).joinKey,
+        (MessageType) ism.getMessage(),
+        ism.getOffset(),
+        ism.getSystemStreamPartition());
+  }
+
+  JsonMessageEnvelope myJoinResult(JsonMessageEnvelope m1, JsonMessageEnvelope m2) {
+    MessageType newJoinMsg = new MessageType();
+    newJoinMsg.joinKey = m1.getKey();
+    newJoinMsg.joinFields.addAll(m1.getMessage().joinFields);
+    newJoinMsg.joinFields.addAll(m2.getMessage().joinFields);
+    return new JsonMessageEnvelope(m1.getMessage().joinKey, newJoinMsg, null, null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamAdaptorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamAdaptorTask.java b/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamAdaptorTask.java
new file mode 100644
index 0000000..61bb32a
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamAdaptorTask.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.samza.operators;
+
+import org.apache.samza.Partition;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.impl.OperatorImpl;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.task.TaskContext;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.lang.reflect.Field;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+
+public class TestFluentStreamAdaptorTask {
+  Field userTaskField = null;
+  Field operatorChainsField = null;
+
+  @Before
+  public void prep() throws NoSuchFieldException {
+    userTaskField = StreamOperatorAdaptorTask.class.getDeclaredField("userTask");
+    operatorChainsField = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
+    userTaskField.setAccessible(true);
+    operatorChainsField.setAccessible(true);
+  }
+
+  @Test
+  public void testConstructor() throws IllegalAccessException {
+    StreamOperatorTask userTask = mock(StreamOperatorTask.class);
+    StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(userTask);
+    StreamOperatorTask taskMemberVar = (StreamOperatorTask) userTaskField.get(adaptorTask);
+    Map<SystemStreamPartition, OperatorImpl> chainsMap = (Map<SystemStreamPartition, OperatorImpl>) operatorChainsField.get(adaptorTask);
+    assertEquals(taskMemberVar, userTask);
+    assertTrue(chainsMap.isEmpty());
+  }
+
+  @Test
+  public void testInit() throws Exception {
+    StreamOperatorTask userTask = mock(StreamOperatorTask.class);
+    StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(userTask);
+    Config mockConfig = mock(Config.class);
+    TaskContext mockContext = mock(TaskContext.class);
+    Set<SystemStreamPartition> testInputs = new HashSet() { {
+        this.add(new SystemStreamPartition("test-sys", "test-strm", new Partition(0)));
+        this.add(new SystemStreamPartition("test-sys", "test-strm", new Partition(1)));
+      } };
+    when(mockContext.getSystemStreamPartitions()).thenReturn(testInputs);
+    adaptorTask.init(mockConfig, mockContext);
+    verify(userTask, times(1)).transform(Mockito.anyMap());
+    Map<SystemStreamPartition, OperatorImpl> chainsMap = (Map<SystemStreamPartition, OperatorImpl>) operatorChainsField.get(adaptorTask);
+    assertTrue(chainsMap.size() == 2);
+    assertTrue(chainsMap.containsKey(testInputs.toArray()[0]));
+    assertTrue(chainsMap.containsKey(testInputs.toArray()[1]));
+  }
+
+  // TODO: window and process methods to be added after implementation of ChainedOperators.create()
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamTasks.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamTasks.java b/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamTasks.java
new file mode 100644
index 0000000..d804bf8
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamTasks.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.samza.operators;
+
+import org.apache.samza.Partition;
+import org.apache.samza.config.Config;
+
+import org.apache.samza.operators.impl.OperatorImpl;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.task.TaskContext;
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
+/**
+ * Unit test for {@link StreamOperatorTask}
+ */
+public class TestFluentStreamTasks {
+
+  private final WindowTask userTask = new WindowTask();
+
+  private final BroadcastTask splitTask = new BroadcastTask();
+
+  private final JoinTask joinTask = new JoinTask();
+
+  private final Set<SystemStreamPartition> inputPartitions = new HashSet<SystemStreamPartition>() { {
+      for (int i = 0; i < 4; i++) {
+        this.add(new SystemStreamPartition("my-system", "my-topic1", new Partition(i)));
+      }
+    } };
+
+  @Test
+  public void testUserTask() throws Exception {
+    Config mockConfig = mock(Config.class);
+    TaskContext mockContext = mock(TaskContext.class);
+    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
+    StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(this.userTask);
+    Field pipelineMapFld = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
+    pipelineMapFld.setAccessible(true);
+    Map<SystemStreamPartition, OperatorImpl> pipelineMap =
+        (Map<SystemStreamPartition, OperatorImpl>) pipelineMapFld.get(adaptorTask);
+
+    adaptorTask.init(mockConfig, mockContext);
+    assertEquals(pipelineMap.size(), 4);
+    this.inputPartitions.forEach(partition -> {
+        assertNotNull(pipelineMap.get(partition));
+      });
+  }
+
+  @Test
+  public void testSplitTask() throws Exception {
+    Config mockConfig = mock(Config.class);
+    TaskContext mockContext = mock(TaskContext.class);
+    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
+    StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(this.splitTask);
+    Field pipelineMapFld = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
+    pipelineMapFld.setAccessible(true);
+    Map<SystemStreamPartition, OperatorImpl> pipelineMap =
+        (Map<SystemStreamPartition, OperatorImpl>) pipelineMapFld.get(adaptorTask);
+
+    adaptorTask.init(mockConfig, mockContext);
+    assertEquals(pipelineMap.size(), 4);
+    this.inputPartitions.forEach(partition -> {
+        assertNotNull(pipelineMap.get(partition));
+      });
+  }
+
+  @Test
+  public void testJoinTask() throws Exception {
+    Config mockConfig = mock(Config.class);
+    TaskContext mockContext = mock(TaskContext.class);
+    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
+    StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(this.joinTask);
+    Field pipelineMapFld = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
+    pipelineMapFld.setAccessible(true);
+    Map<SystemStreamPartition, OperatorImpl> pipelineMap =
+        (Map<SystemStreamPartition, OperatorImpl>) pipelineMapFld.get(adaptorTask);
+
+    adaptorTask.init(mockConfig, mockContext);
+    assertEquals(pipelineMap.size(), 4);
+    this.inputPartitions.forEach(partition -> {
+        assertNotNull(pipelineMap.get(partition));
+      });
+  }
+
+
+}