You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by xi...@apache.org on 2017/10/03 22:23:39 UTC

[01/16] samza git commit: Trigger notification

Repository: samza
Updated Branches:
  refs/heads/master 6e6e53bfd -> db2f121e2


Trigger notification


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

Branch: refs/heads/master
Commit: 6815a4d44da8e9785ae5e9b27ab1e0aeb7dc43a8
Parents: b167963
Author: Xinyu Liu <xi...@xiliu-ld.linkedin.biz>
Authored: Wed Jun 28 14:28:07 2017 -0700
Committer: Xinyu Liu <xi...@xiliu-ld.linkedin.biz>
Committed: Wed Jun 28 14:28:07 2017 -0700

----------------------------------------------------------------------

----------------------------------------------------------------------



[11/16] samza git commit: SAMZA-1386: Inline End-of-stream and Watermark logic inside OperatorImpl

Posted by xi...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/test/java/org/apache/samza/control/TestWatermarkManager.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/control/TestWatermarkManager.java b/samza-core/src/test/java/org/apache/samza/control/TestWatermarkManager.java
deleted file mode 100644
index 8fe7a16..0000000
--- a/samza-core/src/test/java/org/apache/samza/control/TestWatermarkManager.java
+++ /dev/null
@@ -1,260 +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.control;
-
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Multimap;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.apache.samza.Partition;
-import org.apache.samza.container.TaskName;
-import org.apache.samza.message.WatermarkMessage;
-import org.apache.samza.system.IncomingMessageEnvelope;
-import org.apache.samza.system.OutgoingMessageEnvelope;
-import org.apache.samza.system.StreamMetadataCache;
-import org.apache.samza.system.StreamSpec;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.system.SystemStreamMetadata;
-import org.apache.samza.system.SystemStreamPartition;
-import org.apache.samza.task.MessageCollector;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.ArgumentCaptor;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyBoolean;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Matchers.anyObject;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.doNothing;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-
-public class TestWatermarkManager {
-
-  StreamMetadataCache metadataCache;
-
-  @Before
-  public void setup() {
-    SystemStreamMetadata metadata = mock(SystemStreamMetadata.class);
-    Map<Partition, SystemStreamMetadata.SystemStreamPartitionMetadata> partitionMetadata = new HashMap<>();
-    partitionMetadata.put(new Partition(0), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
-    partitionMetadata.put(new Partition(1), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
-    partitionMetadata.put(new Partition(2), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
-    partitionMetadata.put(new Partition(3), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
-    when(metadata.getSystemStreamPartitionMetadata()).thenReturn(partitionMetadata);
-    metadataCache = mock(StreamMetadataCache.class);
-    when(metadataCache.getSystemStreamMetadata(anyObject(), anyBoolean())).thenReturn(metadata);
-  }
-
-  @Test
-  public void testUpdateFromInputSource() {
-    SystemStreamPartition ssp = new SystemStreamPartition("test-system", "test-stream", new Partition(0));
-    TaskName taskName = new TaskName("Task 0");
-    Multimap<SystemStream, String> streamToTasks = HashMultimap.create();
-    streamToTasks.put(ssp.getSystemStream(), taskName.getTaskName());
-    ControlMessageListenerTask listener = mock(ControlMessageListenerTask.class);
-    when(listener.getIOGraph()).thenReturn(new IOGraph(Collections.emptyList()));
-    WatermarkManager manager = new WatermarkManager("Task 0", listener, streamToTasks, Collections.singleton(ssp), null, null);
-    long time = System.currentTimeMillis();
-    Watermark watermark = manager.update(WatermarkManager.buildWatermarkEnvelope(time, ssp));
-    assertEquals(watermark.getTimestamp(), time);
-  }
-
-  @Test
-  public void testUpdateFromIntermediateStream() {
-    SystemStreamPartition[] ssps = new SystemStreamPartition[3];
-    ssps[0] = new SystemStreamPartition("test-system", "test-stream-1", new Partition(0));
-    ssps[1] = new SystemStreamPartition("test-system", "test-stream-2", new Partition(0));
-    ssps[2] = new SystemStreamPartition("test-system", "test-stream-2", new Partition(1));
-
-    TaskName taskName = new TaskName("Task 0");
-    Multimap<SystemStream, String> streamToTasks = HashMultimap.create();
-    for (SystemStreamPartition ssp : ssps) {
-      streamToTasks.put(ssp.getSystemStream(), taskName.getTaskName());
-    }
-    ControlMessageListenerTask listener = mock(ControlMessageListenerTask.class);
-    when(listener.getIOGraph()).thenReturn(new IOGraph(Collections.emptyList()));
-
-    WatermarkManager manager = new WatermarkManager("Task 0", listener, streamToTasks, new HashSet<>(Arrays.asList(ssps)), null, null);
-    int envelopeCount = 4;
-    IncomingMessageEnvelope[] envelopes = new IncomingMessageEnvelope[envelopeCount];
-
-    long[] time = {300L, 200L, 100L, 400L};
-    for (int i = 0; i < envelopeCount; i++) {
-      envelopes[i] = new IncomingMessageEnvelope(ssps[0], "dummy-offset", "", new WatermarkMessage(time[i], "task " + i, envelopeCount));
-    }
-    for (int i = 0; i < 3; i++) {
-      assertNull(manager.update(envelopes[i]));
-    }
-    // verify the first three messages won't result in end-of-stream
-    assertEquals(manager.getWatermarkTime(ssps[0]), WatermarkManager.TIME_NOT_EXIST);
-    // the fourth message will generate a watermark
-    Watermark watermark = manager.update(envelopes[3]);
-    assertNotNull(watermark);
-    assertEquals(watermark.getTimestamp(), 100);
-    assertEquals(manager.getWatermarkTime(ssps[1]), WatermarkManager.TIME_NOT_EXIST);
-    assertEquals(manager.getWatermarkTime(ssps[2]), WatermarkManager.TIME_NOT_EXIST);
-
-
-    // stream2 has two partitions assigned to this task, so it requires a message from each partition to calculate watermarks
-    long[] time1 = {300L, 200L, 100L, 400L};
-    envelopes = new IncomingMessageEnvelope[envelopeCount];
-    for (int i = 0; i < envelopeCount; i++) {
-      envelopes[i] = new IncomingMessageEnvelope(ssps[1], "dummy-offset", "", new WatermarkMessage(time1[i], "task " + i, envelopeCount));
-    }
-    // verify the messages for the partition 0 won't generate watermark
-    for (int i = 0; i < 4; i++) {
-      assertNull(manager.update(envelopes[i]));
-    }
-    assertEquals(manager.getWatermarkTime(ssps[1]), 100L);
-
-    long[] time2 = {350L, 150L, 500L, 80L};
-    for (int i = 0; i < envelopeCount; i++) {
-      envelopes[i] = new IncomingMessageEnvelope(ssps[2], "dummy-offset", "", new WatermarkMessage(time2[i], "task " + i, envelopeCount));
-    }
-    for (int i = 0; i < 3; i++) {
-      assertNull(manager.update(envelopes[i]));
-    }
-    assertEquals(manager.getWatermarkTime(ssps[2]), WatermarkManager.TIME_NOT_EXIST);
-    // the fourth message will generate the watermark
-    watermark = manager.update(envelopes[3]);
-    assertNotNull(watermark);
-    assertEquals(manager.getWatermarkTime(ssps[2]), 80L);
-    assertEquals(watermark.getTimestamp(), 80L);
-  }
-
-  @Test
-  public void testSendWatermark() {
-    SystemStream ints = new SystemStream("test-system", "int-stream");
-    SystemStreamMetadata metadata = mock(SystemStreamMetadata.class);
-    Map<Partition, SystemStreamMetadata.SystemStreamPartitionMetadata> partitionMetadata = new HashMap<>();
-    partitionMetadata.put(new Partition(0), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
-    partitionMetadata.put(new Partition(1), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
-    partitionMetadata.put(new Partition(2), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
-    partitionMetadata.put(new Partition(3), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
-    when(metadata.getSystemStreamPartitionMetadata()).thenReturn(partitionMetadata);
-    StreamMetadataCache metadataCache = mock(StreamMetadataCache.class);
-    when(metadataCache.getSystemStreamMetadata(anyObject(), anyBoolean())).thenReturn(metadata);
-
-    MessageCollector collector = mock(MessageCollector.class);
-    ControlMessageListenerTask listener = mock(ControlMessageListenerTask.class);
-    when(listener.getIOGraph()).thenReturn(new IOGraph(Collections.emptyList()));
-
-    WatermarkManager manager = new WatermarkManager("task 0",
-        listener,
-        HashMultimap.create(),
-        Collections.EMPTY_SET,
-        metadataCache,
-        collector);
-
-    long time = System.currentTimeMillis();
-    Set<Integer> partitions = new HashSet<>();
-    doAnswer(invocation -> {
-        OutgoingMessageEnvelope envelope = (OutgoingMessageEnvelope) invocation.getArguments()[0];
-        partitions.add((Integer) envelope.getPartitionKey());
-        WatermarkMessage watermarkMessage = (WatermarkMessage) envelope.getMessage();
-        assertEquals(watermarkMessage.getTaskName(), "task 0");
-        assertEquals(watermarkMessage.getTaskCount(), 8);
-        assertEquals(watermarkMessage.getTimestamp(), time);
-        return null;
-      }).when(collector).send(any());
-
-    manager.sendWatermark(time, ints, 8);
-    assertEquals(partitions.size(), 4);
-  }
-
-  @Test
-  public void testPropagate() {
-    StreamSpec outputSpec = new StreamSpec("int-stream", "int-stream", "test-system");
-    List<StreamSpec> inputs = new ArrayList<>();
-    inputs.add(new StreamSpec("input-stream-1", "input-stream-1", "test-system"));
-    inputs.add(new StreamSpec("input-stream-2", "input-stream-2", "test-system"));
-
-    IOGraph ioGraph = TestIOGraph.buildSimpleIOGraph(inputs, outputSpec, true);
-
-    SystemStream input1 = new SystemStream("test-system", "input-stream-1");
-    SystemStream input2 = new SystemStream("test-system", "input-stream-2");
-    SystemStream input3 = new SystemStream("test-system", "input-stream-3");
-    SystemStream ints = new SystemStream("test-system", "int-stream");
-    SystemStreamPartition[] ssps0 = new SystemStreamPartition[3];
-    ssps0[0] = new SystemStreamPartition(input1, new Partition(0));
-    ssps0[1] = new SystemStreamPartition(input2, new Partition(0));
-    ssps0[2] = new SystemStreamPartition(ints, new Partition(0));
-
-    SystemStreamPartition[] ssps1 = new SystemStreamPartition[4];
-    ssps1[0] = new SystemStreamPartition(input1, new Partition(1));
-    ssps1[1] = new SystemStreamPartition(input2, new Partition(1));
-    ssps1[2] = new SystemStreamPartition(input3, new Partition(1));
-    ssps1[3] = new SystemStreamPartition(ints, new Partition(1));
-
-    SystemStreamPartition[] ssps2 = new SystemStreamPartition[2];
-    ssps2[0] = new SystemStreamPartition(input3, new Partition(2));
-    ssps2[1] = new SystemStreamPartition(ints, new Partition(2));
-
-
-    TaskName t0 = new TaskName("task 0"); //consume input1 and input2
-    TaskName t1 = new TaskName("task 1"); //consume input 1 and input2 and input 3
-    TaskName t2 = new TaskName("task 2"); //consume input2 and input 3
-    Multimap<SystemStream, String> inputToTasks = HashMultimap.create();
-    for (SystemStreamPartition ssp : ssps0) {
-      inputToTasks.put(ssp.getSystemStream(), t0.getTaskName());
-    }
-    for (SystemStreamPartition ssp : ssps1) {
-      inputToTasks.put(ssp.getSystemStream(), t1.getTaskName());
-    }
-    for (SystemStreamPartition ssp : ssps2) {
-      inputToTasks.put(ssp.getSystemStream(), t2.getTaskName());
-    }
-
-    ControlMessageListenerTask listener = mock(ControlMessageListenerTask.class);
-    when(listener.getIOGraph()).thenReturn(ioGraph);
-    WatermarkManager manager = spy(
-        new WatermarkManager(t0.getTaskName(), listener, inputToTasks, new HashSet<>(Arrays.asList(ssps0)), null, null));
-
-    IncomingMessageEnvelope envelope = WatermarkManager.buildWatermarkEnvelope(System.currentTimeMillis(), ssps0[0]);
-    doNothing().when(manager).sendWatermark(anyLong(), any(), anyInt());
-    Watermark watermark = manager.update(envelope);
-    assertNotNull(watermark);
-    long time = System.currentTimeMillis();
-    Watermark updatedWatermark = watermark.copyWithTimestamp(time);
-    updatedWatermark.propagate(ints);
-    ArgumentCaptor<Long> arg1 = ArgumentCaptor.forClass(Long.class);
-    ArgumentCaptor<SystemStream> arg2 = ArgumentCaptor.forClass(SystemStream.class);
-    ArgumentCaptor<Integer> arg3 = ArgumentCaptor.forClass(Integer.class);
-    verify(manager).sendWatermark(arg1.capture(), arg2.capture(), arg3.capture());
-    assertEquals(arg1.getValue().longValue(), time);
-    assertEquals(arg2.getValue(), ints);
-    assertEquals(arg3.getValue().intValue(), 2);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java b/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java
index c51b1ea..af0b786 100644
--- a/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java
+++ b/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java
@@ -32,6 +32,7 @@ import org.apache.samza.system.SystemStream;
 import org.apache.samza.system.SystemStreamPartition;
 import org.apache.samza.task.MessageCollector;
 import org.apache.samza.task.StreamOperatorTask;
+import org.apache.samza.container.TaskContextImpl;
 import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
 import org.apache.samza.testUtils.TestClock;
@@ -253,7 +254,7 @@ public class TestJoinOperator {
     when(runner.getStreamSpec("instream")).thenReturn(new StreamSpec("instream", "instream", "insystem"));
     when(runner.getStreamSpec("instream2")).thenReturn(new StreamSpec("instream2", "instream2", "insystem2"));
 
-    TaskContext taskContext = mock(TaskContext.class);
+    TaskContextImpl taskContext = mock(TaskContextImpl.class);
     when(taskContext.getSystemStreamPartitions()).thenReturn(ImmutableSet
         .of(new SystemStreamPartition("insystem", "instream", new Partition(0)),
             new SystemStreamPartition("insystem2", "instream2", new Partition(0))));

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/test/java/org/apache/samza/operators/TestWindowOperator.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestWindowOperator.java b/samza-core/src/test/java/org/apache/samza/operators/TestWindowOperator.java
index ca8a151..1edc5f6 100644
--- a/samza-core/src/test/java/org/apache/samza/operators/TestWindowOperator.java
+++ b/samza-core/src/test/java/org/apache/samza/operators/TestWindowOperator.java
@@ -26,6 +26,7 @@ import junit.framework.Assert;
 import org.apache.samza.Partition;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.Config;
+import org.apache.samza.container.TaskContextImpl;
 import org.apache.samza.metrics.MetricsRegistryMap;
 import org.apache.samza.operators.triggers.FiringType;
 import org.apache.samza.operators.triggers.Trigger;
@@ -41,7 +42,6 @@ import org.apache.samza.system.SystemStream;
 import org.apache.samza.system.SystemStreamPartition;
 import org.apache.samza.task.MessageCollector;
 import org.apache.samza.task.StreamOperatorTask;
-import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
 import org.apache.samza.testUtils.TestClock;
 import org.junit.Before;
@@ -60,13 +60,13 @@ public class TestWindowOperator {
   private final TaskCoordinator taskCoordinator = mock(TaskCoordinator.class);
   private final List<Integer> integers = ImmutableList.of(1, 2, 1, 2, 1, 2, 1, 2, 3);
   private Config config;
-  private TaskContext taskContext;
+  private TaskContextImpl taskContext;
   private ApplicationRunner runner;
 
   @Before
   public void setup() throws Exception {
     config = mock(Config.class);
-    taskContext = mock(TaskContext.class);
+    taskContext = mock(TaskContextImpl.class);
     runner = mock(ApplicationRunner.class);
     when(taskContext.getSystemStreamPartitions()).thenReturn(ImmutableSet
         .of(new SystemStreamPartition("kafka", "integers", new Partition(0))));

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/test/java/org/apache/samza/operators/impl/TestControlMessageSender.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestControlMessageSender.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestControlMessageSender.java
new file mode 100644
index 0000000..d17d751
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestControlMessageSender.java
@@ -0,0 +1,73 @@
+/*
+ * 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 java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import org.apache.samza.Partition;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.StreamMetadataCache;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamMetadata;
+import org.apache.samza.system.WatermarkMessage;
+import org.apache.samza.task.MessageCollector;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
+public class TestControlMessageSender {
+
+  @Test
+  public void testSend() {
+    SystemStreamMetadata metadata = mock(SystemStreamMetadata.class);
+    Map<Partition, SystemStreamMetadata.SystemStreamPartitionMetadata> partitionMetadata = new HashMap<>();
+    partitionMetadata.put(new Partition(0), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
+    partitionMetadata.put(new Partition(1), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
+    partitionMetadata.put(new Partition(2), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
+    partitionMetadata.put(new Partition(3), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
+    when(metadata.getSystemStreamPartitionMetadata()).thenReturn(partitionMetadata);
+    StreamMetadataCache metadataCache = mock(StreamMetadataCache.class);
+    when(metadataCache.getSystemStreamMetadata(anyObject(), anyBoolean())).thenReturn(metadata);
+
+    SystemStream systemStream = new SystemStream("test-system", "test-stream");
+    Set<Integer> partitions = new HashSet<>();
+    MessageCollector collector = mock(MessageCollector.class);
+    doAnswer(invocation -> {
+        OutgoingMessageEnvelope envelope = (OutgoingMessageEnvelope) invocation.getArguments()[0];
+        partitions.add((Integer) envelope.getPartitionKey());
+        assertEquals(envelope.getSystemStream(), systemStream);
+        return null;
+      }).when(collector).send(any());
+
+    ControlMessageSender sender = new ControlMessageSender(metadataCache);
+    WatermarkMessage watermark = new WatermarkMessage(System.currentTimeMillis(), "task 0");
+    sender.send(watermark, systemStream, collector);
+    assertEquals(partitions.size(), 4);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/test/java/org/apache/samza/operators/impl/TestEndOfStreamStates.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestEndOfStreamStates.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestEndOfStreamStates.java
new file mode 100644
index 0000000..887991f
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestEndOfStreamStates.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.operators.impl;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import org.apache.samza.Partition;
+import org.apache.samza.system.EndOfStreamMessage;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+import org.junit.Test;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+
+public class TestEndOfStreamStates {
+
+  @Test
+  public void testUpdate() {
+    SystemStream input = new SystemStream("system", "input");
+    SystemStream intermediate = new SystemStream("system", "intermediate");
+
+    Set<SystemStreamPartition> ssps = new HashSet<>();
+    SystemStreamPartition inputPartition0 = new SystemStreamPartition(input, new Partition(0));
+    SystemStreamPartition intPartition0 = new SystemStreamPartition(intermediate, new Partition(0));
+    SystemStreamPartition intPartition1 = new SystemStreamPartition(intermediate, new Partition(1));
+    ssps.add(inputPartition0);
+    ssps.add(intPartition0);
+    ssps.add(intPartition1);
+
+    Map<SystemStream, Integer> producerCounts = new HashMap<>();
+    producerCounts.put(intermediate, 2);
+
+    EndOfStreamStates endOfStreamStates = new EndOfStreamStates(ssps, producerCounts);
+    assertFalse(endOfStreamStates.isEndOfStream(input));
+    assertFalse(endOfStreamStates.isEndOfStream(intermediate));
+    assertFalse(endOfStreamStates.allEndOfStream());
+
+    IncomingMessageEnvelope envelope = IncomingMessageEnvelope.buildEndOfStreamEnvelope(inputPartition0);
+    endOfStreamStates.update((EndOfStreamMessage) envelope.getMessage(), envelope.getSystemStreamPartition());
+    assertTrue(endOfStreamStates.isEndOfStream(input));
+    assertFalse(endOfStreamStates.isEndOfStream(intermediate));
+    assertFalse(endOfStreamStates.allEndOfStream());
+
+    EndOfStreamMessage eos = new EndOfStreamMessage("task 0");
+    endOfStreamStates.update(eos, intPartition0);
+    endOfStreamStates.update(eos, intPartition1);
+    assertFalse(endOfStreamStates.isEndOfStream(intermediate));
+    assertFalse(endOfStreamStates.allEndOfStream());
+
+    eos = new EndOfStreamMessage("task 1");
+    endOfStreamStates.update(eos, intPartition0);
+    endOfStreamStates.update(eos, intPartition1);
+    assertTrue(endOfStreamStates.isEndOfStream(intermediate));
+    assertTrue(endOfStreamStates.allEndOfStream());
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
index 3ae8f5b..4a78da8 100644
--- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
+++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
@@ -18,21 +18,22 @@
  */
 package org.apache.samza.operators.impl;
 
+import java.util.Collection;
+import java.util.Collections;
 import java.util.Set;
 import org.apache.samza.config.Config;
+import org.apache.samza.container.TaskContextImpl;
 import org.apache.samza.metrics.Counter;
-import org.apache.samza.metrics.MetricsRegistry;
 import org.apache.samza.metrics.MetricsRegistryMap;
+import org.apache.samza.metrics.ReadableMetricsRegistry;
 import org.apache.samza.metrics.Timer;
+import org.apache.samza.operators.functions.WatermarkFunction;
 import org.apache.samza.operators.spec.OperatorSpec;
 import org.apache.samza.task.MessageCollector;
 import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
 import org.junit.Test;
 
-import java.util.Collection;
-import java.util.Collections;
-
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.anyObject;
 import static org.mockito.Matchers.anyString;
@@ -47,7 +48,7 @@ public class TestOperatorImpl {
   @Test(expected = IllegalStateException.class)
   public void testMultipleInitShouldThrow() {
     OperatorImpl<Object, Object> opImpl = new TestOpImpl(mock(Object.class));
-    TaskContext mockTaskContext = mock(TaskContext.class);
+    TaskContextImpl mockTaskContext = mock(TaskContextImpl.class);
     when(mockTaskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap());
     opImpl.init(mock(Config.class), mockTaskContext);
     opImpl.init(mock(Config.class), mockTaskContext);
@@ -61,7 +62,7 @@ public class TestOperatorImpl {
 
   @Test
   public void testOnMessagePropagatesResults() {
-    TaskContext mockTaskContext = mock(TaskContext.class);
+    TaskContextImpl mockTaskContext = mock(TaskContextImpl.class);
     when(mockTaskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap());
 
     Object mockTestOpImplOutput = mock(Object.class);
@@ -93,8 +94,8 @@ public class TestOperatorImpl {
 
   @Test
   public void testOnMessageUpdatesMetrics() {
-    TaskContext mockTaskContext = mock(TaskContext.class);
-    MetricsRegistry mockMetricsRegistry = mock(MetricsRegistry.class);
+    TaskContextImpl mockTaskContext = mock(TaskContextImpl.class);
+    ReadableMetricsRegistry mockMetricsRegistry = mock(ReadableMetricsRegistry.class);
     when(mockTaskContext.getMetricsRegistry()).thenReturn(mockMetricsRegistry);
     Counter mockCounter = mock(Counter.class);
     Timer mockTimer = mock(Timer.class);
@@ -117,7 +118,7 @@ public class TestOperatorImpl {
 
   @Test
   public void testOnTimerPropagatesResultsAndTimer() {
-    TaskContext mockTaskContext = mock(TaskContext.class);
+    TaskContextImpl mockTaskContext = mock(TaskContextImpl.class);
     when(mockTaskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap());
 
     Object mockTestOpImplOutput = mock(Object.class);
@@ -153,8 +154,8 @@ public class TestOperatorImpl {
 
   @Test
   public void testOnTimerUpdatesMetrics() {
-    TaskContext mockTaskContext = mock(TaskContext.class);
-    MetricsRegistry mockMetricsRegistry = mock(MetricsRegistry.class);
+    TaskContextImpl mockTaskContext = mock(TaskContextImpl.class);
+    ReadableMetricsRegistry mockMetricsRegistry = mock(ReadableMetricsRegistry.class);
     when(mockTaskContext.getMetricsRegistry()).thenReturn(mockMetricsRegistry);
     Counter mockMessageCounter = mock(Counter.class);
     Timer mockTimer = mock(Timer.class);
@@ -210,6 +211,11 @@ public class TestOperatorImpl {
     TestOpSpec() {
      super(OpCode.INPUT, 1);
     }
+
+    @Override
+    public WatermarkFunction getWatermarkFn() {
+      return null;
+    }
   }
 
   public static Set<OperatorImpl> getNextOperators(OperatorImpl op) {
@@ -221,11 +227,11 @@ public class TestOperatorImpl {
   }
 
   public static long getInputWatermark(OperatorImpl op) {
-    return op.getInputWatermarkTime();
+    return op.getInputWatermark();
   }
 
   public static long getOutputWatermark(OperatorImpl op) {
-    return op.getOutputWatermarkTime();
+    return op.getOutputWatermark();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java
index 4505eef..9fab1b7 100644
--- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java
+++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java
@@ -19,8 +19,23 @@
 
 package org.apache.samza.operators.impl;
 
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.samza.Partition;
 import org.apache.samza.config.Config;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.container.TaskContextImpl;
+import org.apache.samza.container.TaskName;
+import org.apache.samza.job.model.ContainerModel;
+import org.apache.samza.job.model.JobModel;
+import org.apache.samza.job.model.TaskModel;
 import org.apache.samza.metrics.MetricsRegistryMap;
 import org.apache.samza.operators.MessageStream;
 import org.apache.samza.operators.OutputStream;
@@ -32,6 +47,7 @@ import org.apache.samza.operators.spec.OperatorSpec.OpCode;
 import org.apache.samza.runtime.ApplicationRunner;
 import org.apache.samza.system.StreamSpec;
 import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
 import org.apache.samza.task.MessageCollector;
 import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
@@ -62,7 +78,7 @@ public class TestOperatorImplGraph {
   public void testEmptyChain() {
     StreamGraphImpl streamGraph = new StreamGraphImpl(mock(ApplicationRunner.class), mock(Config.class));
     OperatorImplGraph opGraph =
-        new OperatorImplGraph(streamGraph, mock(Config.class), mock(TaskContext.class), mock(Clock.class));
+        new OperatorImplGraph(streamGraph, mock(Config.class), mock(TaskContextImpl.class), mock(Clock.class));
     assertEquals(0, opGraph.getAllInputOperators().size());
   }
 
@@ -82,8 +98,9 @@ public class TestOperatorImplGraph {
         .map(mock(MapFunction.class))
         .sendTo(outputStream);
 
-    TaskContext mockTaskContext = mock(TaskContext.class);
+    TaskContextImpl mockTaskContext = mock(TaskContextImpl.class);
     when(mockTaskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap());
+    when(mockTaskContext.getTaskName()).thenReturn(new TaskName("task 0"));
     OperatorImplGraph opImplGraph =
         new OperatorImplGraph(streamGraph, mock(Config.class), mockTaskContext, mock(Clock.class));
 
@@ -113,17 +130,17 @@ public class TestOperatorImplGraph {
     inputStream.filter(mock(FilterFunction.class));
     inputStream.map(mock(MapFunction.class));
 
-    TaskContext mockTaskContext = mock(TaskContext.class);
+    TaskContextImpl mockTaskContext = mock(TaskContextImpl.class);
     when(mockTaskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap());
     OperatorImplGraph opImplGraph =
         new OperatorImplGraph(streamGraph, mock(Config.class), mockTaskContext, mock(Clock.class));
 
     InputOperatorImpl inputOpImpl = opImplGraph.getInputOperator(new SystemStream("input-system", "input-stream"));
     assertEquals(2, inputOpImpl.registeredOperators.size());
-    assertTrue(inputOpImpl.registeredOperators.stream().anyMatch(opImpl ->
-        ((OperatorImpl) opImpl).getOperatorSpec().getOpCode() == OpCode.FILTER));
-    assertTrue(inputOpImpl.registeredOperators.stream().anyMatch(opImpl ->
-        ((OperatorImpl) opImpl).getOperatorSpec().getOpCode() == OpCode.MAP));
+    assertTrue(inputOpImpl.registeredOperators.stream()
+        .anyMatch(opImpl -> ((OperatorImpl) opImpl).getOperatorSpec().getOpCode() == OpCode.FILTER));
+    assertTrue(inputOpImpl.registeredOperators.stream()
+        .anyMatch(opImpl -> ((OperatorImpl) opImpl).getOperatorSpec().getOpCode() == OpCode.MAP));
   }
 
   @Test
@@ -139,13 +156,13 @@ public class TestOperatorImplGraph {
     MapFunction mockMapFunction = mock(MapFunction.class);
     mergedStream.map(mockMapFunction);
 
-    TaskContext mockTaskContext = mock(TaskContext.class);
+    TaskContextImpl mockTaskContext = mock(TaskContextImpl.class);
     when(mockTaskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap());
     OperatorImplGraph opImplGraph =
         new OperatorImplGraph(streamGraph, mock(Config.class), mockTaskContext, mock(Clock.class));
 
     // verify that the DAG after merge is only traversed & initialized once
-    verify(mockMapFunction, times(1)).init(any(Config.class), any(TaskContext.class));
+    verify(mockMapFunction, times(1)).init(any(Config.class), any(TaskContextImpl.class));
   }
 
   @Test
@@ -160,13 +177,13 @@ public class TestOperatorImplGraph {
     MessageStream<Object> inputStream2 = streamGraph.getInputStream("input2", (k, v) -> v);
     inputStream1.join(inputStream2, mockJoinFunction, Duration.ofHours(1));
 
-    TaskContext mockTaskContext = mock(TaskContext.class);
+    TaskContextImpl mockTaskContext = mock(TaskContextImpl.class);
     when(mockTaskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap());
     OperatorImplGraph opImplGraph =
         new OperatorImplGraph(streamGraph, mock(Config.class), mockTaskContext, mock(Clock.class));
 
     // verify that join function is initialized once.
-    verify(mockJoinFunction, times(1)).init(any(Config.class), any(TaskContext.class));
+    verify(mockJoinFunction, times(1)).init(any(Config.class), any(TaskContextImpl.class));
 
     InputOperatorImpl inputOpImpl1 = opImplGraph.getInputOperator(new SystemStream("input-system", "input-stream1"));
     InputOperatorImpl inputOpImpl2 = opImplGraph.getInputOperator(new SystemStream("input-system", "input-stream2"));
@@ -201,7 +218,7 @@ public class TestOperatorImplGraph {
     when(mockRunner.getStreamSpec("input1")).thenReturn(new StreamSpec("input1", "input-stream1", "input-system"));
     when(mockRunner.getStreamSpec("input2")).thenReturn(new StreamSpec("input2", "input-stream2", "input-system"));
     Config mockConfig = mock(Config.class);
-    TaskContext mockContext = mock(TaskContext.class);
+    TaskContextImpl mockContext = mock(TaskContextImpl.class);
     when(mockContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap());
     StreamGraphImpl streamGraph = new StreamGraphImpl(mockRunner, mockConfig);
 
@@ -255,4 +272,144 @@ public class TestOperatorImplGraph {
       }
     };
   }
+
+  @Test
+  public void testGetStreamToConsumerTasks() {
+    String system = "test-system";
+    String stream0 = "test-stream-0";
+    String stream1 = "test-stream-1";
+
+    SystemStreamPartition ssp0 = new SystemStreamPartition(system, stream0, new Partition(0));
+    SystemStreamPartition ssp1 = new SystemStreamPartition(system, stream0, new Partition(1));
+    SystemStreamPartition ssp2 = new SystemStreamPartition(system, stream1, new Partition(0));
+
+    TaskName task0 = new TaskName("Task 0");
+    TaskName task1 = new TaskName("Task 1");
+    Set<SystemStreamPartition> ssps = new HashSet<>();
+    ssps.add(ssp0);
+    ssps.add(ssp2);
+    TaskModel tm0 = new TaskModel(task0, ssps, new Partition(0));
+    ContainerModel cm0 = new ContainerModel("c0", 0, Collections.singletonMap(task0, tm0));
+    TaskModel tm1 = new TaskModel(task1, Collections.singleton(ssp1), new Partition(1));
+    ContainerModel cm1 = new ContainerModel("c1", 1, Collections.singletonMap(task1, tm1));
+
+    Map<String, ContainerModel> cms = new HashMap<>();
+    cms.put(cm0.getProcessorId(), cm0);
+    cms.put(cm1.getProcessorId(), cm1);
+
+    JobModel jobModel = new JobModel(new MapConfig(), cms, null);
+    Multimap<SystemStream, String> streamToTasks = OperatorImplGraph.getStreamToConsumerTasks(jobModel);
+    assertEquals(streamToTasks.get(ssp0.getSystemStream()).size(), 2);
+    assertEquals(streamToTasks.get(ssp2.getSystemStream()).size(), 1);
+  }
+
+  @Test
+  public void testGetOutputToInputStreams() {
+    Map<String, String> configMap = new HashMap<>();
+    configMap.put(JobConfig.JOB_NAME(), "test-app");
+    configMap.put(JobConfig.JOB_DEFAULT_SYSTEM(), "test-system");
+    Config config = new MapConfig(configMap);
+
+    /**
+     * the graph looks like the following. number of partitions in parentheses. quotes indicate expected value.
+     *
+     *                                    input1 -> map -> join -> partitionBy (10) -> output1
+     *                                                       |
+     *                                     input2 -> filter -|
+     *                                                       |
+     *           input3 -> filter -> partitionBy -> map -> join -> output2
+     *
+     */
+    StreamSpec input1 = new StreamSpec("input1", "input1", "system1");
+    StreamSpec input2 = new StreamSpec("input2", "input2", "system2");
+    StreamSpec input3 = new StreamSpec("input3", "input3", "system2");
+
+    StreamSpec output1 = new StreamSpec("output1", "output1", "system1");
+    StreamSpec output2 = new StreamSpec("output2", "output2", "system2");
+
+    ApplicationRunner runner = mock(ApplicationRunner.class);
+    when(runner.getStreamSpec("input1")).thenReturn(input1);
+    when(runner.getStreamSpec("input2")).thenReturn(input2);
+    when(runner.getStreamSpec("input3")).thenReturn(input3);
+    when(runner.getStreamSpec("output1")).thenReturn(output1);
+    when(runner.getStreamSpec("output2")).thenReturn(output2);
+
+    // intermediate streams used in tests
+    StreamSpec int1 = new StreamSpec("test-app-1-partition_by-10", "test-app-1-partition_by-10", "default-system");
+    StreamSpec int2 = new StreamSpec("test-app-1-partition_by-6", "test-app-1-partition_by-6", "default-system");
+    when(runner.getStreamSpec("test-app-1-partition_by-10"))
+        .thenReturn(int1);
+    when(runner.getStreamSpec("test-app-1-partition_by-6"))
+        .thenReturn(int2);
+
+    StreamGraphImpl streamGraph = new StreamGraphImpl(runner, config);
+    BiFunction msgBuilder = mock(BiFunction.class);
+    MessageStream m1 = streamGraph.getInputStream("input1", msgBuilder).map(m -> m);
+    MessageStream m2 = streamGraph.getInputStream("input2", msgBuilder).filter(m -> true);
+    MessageStream m3 = streamGraph.getInputStream("input3", msgBuilder).filter(m -> true).partitionBy(m -> "hehe").map(m -> m);
+    Function mockFn = mock(Function.class);
+    OutputStream<Object, Object, Object> om1 = streamGraph.getOutputStream("output1", mockFn, mockFn);
+    OutputStream<Object, Object, Object> om2 = streamGraph.getOutputStream("output2", mockFn, mockFn);
+
+    m1.join(m2, mock(JoinFunction.class), Duration.ofHours(2)).partitionBy(m -> "haha").sendTo(om1);
+    m3.join(m2, mock(JoinFunction.class), Duration.ofHours(1)).sendTo(om2);
+
+    Multimap<SystemStream, SystemStream> outputToInput = OperatorImplGraph.getIntermediateToInputStreamsMap(streamGraph);
+    Collection<SystemStream> inputs = outputToInput.get(int1.toSystemStream());
+    assertEquals(inputs.size(), 2);
+    assertTrue(inputs.contains(input1.toSystemStream()));
+    assertTrue(inputs.contains(input2.toSystemStream()));
+
+    inputs = outputToInput.get(int2.toSystemStream());
+    assertEquals(inputs.size(), 1);
+    assertEquals(inputs.iterator().next(), input3.toSystemStream());
+  }
+
+  @Test
+  public void testGetProducerTaskCountForIntermediateStreams() {
+    /**
+     * the task assignment looks like the following:
+     *
+     * input1 -----> task0, task1 -----> int1
+     *                                    ^
+     * input2 ------> task1, task2--------|
+     *                                    v
+     * input3 ------> task1 -----------> int2
+     *
+     */
+
+    SystemStream input1 = new SystemStream("system1", "intput1");
+    SystemStream input2 = new SystemStream("system2", "intput2");
+    SystemStream input3 = new SystemStream("system2", "intput3");
+
+    SystemStream int1 = new SystemStream("system1", "int1");
+    SystemStream int2 = new SystemStream("system1", "int2");
+
+
+    String task0 = "Task 0";
+    String task1 = "Task 1";
+    String task2 = "Task 2";
+
+    Multimap<SystemStream, String> streamToConsumerTasks = HashMultimap.create();
+    streamToConsumerTasks.put(input1, task0);
+    streamToConsumerTasks.put(input1, task1);
+    streamToConsumerTasks.put(input2, task1);
+    streamToConsumerTasks.put(input2, task2);
+    streamToConsumerTasks.put(input3, task1);
+    streamToConsumerTasks.put(int1, task0);
+    streamToConsumerTasks.put(int1, task1);
+    streamToConsumerTasks.put(int2, task0);
+
+    Multimap<SystemStream, SystemStream> intermediateToInputStreams = HashMultimap.create();
+    intermediateToInputStreams.put(int1, input1);
+    intermediateToInputStreams.put(int1, input2);
+
+    intermediateToInputStreams.put(int2, input2);
+    intermediateToInputStreams.put(int2, input3);
+
+    Map<SystemStream, Integer> counts = OperatorImplGraph.getProducerTaskCountForIntermediateStreams(
+        streamToConsumerTasks, intermediateToInputStreams);
+    assertTrue(counts.get(int1) == 3);
+    assertTrue(counts.get(int2) == 2);
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/test/java/org/apache/samza/operators/impl/TestWatermarkStates.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestWatermarkStates.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestWatermarkStates.java
new file mode 100644
index 0000000..a726069
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestWatermarkStates.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.impl;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import org.apache.samza.Partition;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.system.WatermarkMessage;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+import static org.apache.samza.operators.impl.WatermarkStates.WATERMARK_NOT_EXIST;
+
+public class TestWatermarkStates {
+
+  @Test
+  public void testUpdate() {
+    SystemStream input = new SystemStream("system", "input");
+    SystemStream intermediate = new SystemStream("system", "intermediate");
+
+    Set<SystemStreamPartition> ssps = new HashSet<>();
+    SystemStreamPartition inputPartition0 = new SystemStreamPartition(input, new Partition(0));
+    SystemStreamPartition intPartition0 = new SystemStreamPartition(intermediate, new Partition(0));
+    SystemStreamPartition intPartition1 = new SystemStreamPartition(intermediate, new Partition(1));
+    ssps.add(inputPartition0);
+    ssps.add(intPartition0);
+    ssps.add(intPartition1);
+
+    Map<SystemStream, Integer> producerCounts = new HashMap<>();
+    producerCounts.put(intermediate, 2);
+
+    // advance watermark on input to 5
+    WatermarkStates watermarkStates = new WatermarkStates(ssps, producerCounts);
+    IncomingMessageEnvelope envelope = IncomingMessageEnvelope.buildWatermarkEnvelope(inputPartition0, 5L);
+    watermarkStates.update((WatermarkMessage) envelope.getMessage(),
+        envelope.getSystemStreamPartition());
+    assertEquals(watermarkStates.getWatermark(input), 5L);
+    assertEquals(watermarkStates.getWatermark(intermediate), WATERMARK_NOT_EXIST);
+
+    // watermark from task 0 on int p0 to 6
+    WatermarkMessage watermarkMessage = new WatermarkMessage(6L, "task 0");
+    watermarkStates.update(watermarkMessage, intPartition0);
+    assertEquals(watermarkStates.getWatermarkPerSSP(intPartition0), WATERMARK_NOT_EXIST);
+    assertEquals(watermarkStates.getWatermark(intermediate), WATERMARK_NOT_EXIST);
+
+    // watermark from task 1 on int p0 to 3
+    watermarkMessage = new WatermarkMessage(3L, "task 1");
+    watermarkStates.update(watermarkMessage, intPartition0);
+    assertEquals(watermarkStates.getWatermarkPerSSP(intPartition0), 3L);
+    assertEquals(watermarkStates.getWatermark(intermediate), WATERMARK_NOT_EXIST);
+
+    // watermark from task 0 on int p1 to 10
+    watermarkMessage = new WatermarkMessage(10L, "task 0");
+    watermarkStates.update(watermarkMessage, intPartition1);
+    assertEquals(watermarkStates.getWatermarkPerSSP(intPartition1), WATERMARK_NOT_EXIST);
+    assertEquals(watermarkStates.getWatermark(intermediate), WATERMARK_NOT_EXIST);
+
+    // watermark from task 1 on int p1 to 4
+    watermarkMessage = new WatermarkMessage(4L, "task 1");
+    watermarkStates.update(watermarkMessage, intPartition1);
+    assertEquals(watermarkStates.getWatermarkPerSSP(intPartition1), 4L);
+    // verify we got a watermark 3 (min) for int stream
+    assertEquals(watermarkStates.getWatermark(intermediate), 3L);
+
+    // advance watermark from task 1 on int p0 to 8
+    watermarkMessage = new WatermarkMessage(8L, "task 1");
+    watermarkStates.update(watermarkMessage, intPartition0);
+    assertEquals(watermarkStates.getWatermarkPerSSP(intPartition0), 6L);
+    // verify we got a watermark 4 (min) for int stream
+    assertEquals(watermarkStates.getWatermark(intermediate), 4L);
+
+    // advance watermark from task 1 on int p1 to 7
+    watermarkMessage = new WatermarkMessage(7L, "task 1");
+    watermarkStates.update(watermarkMessage, intPartition1);
+    assertEquals(watermarkStates.getWatermarkPerSSP(intPartition1), 7L);
+    // verify we got a watermark 6 (min) for int stream
+    assertEquals(watermarkStates.getWatermark(intermediate), 6L);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestIntermediateMessageSerde.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestIntermediateMessageSerde.java b/samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestIntermediateMessageSerde.java
index 7192525..7a3faca 100644
--- a/samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestIntermediateMessageSerde.java
+++ b/samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestIntermediateMessageSerde.java
@@ -24,11 +24,11 @@ import java.io.ByteArrayOutputStream;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
 import java.io.Serializable;
-import org.apache.samza.message.EndOfStreamMessage;
-import org.apache.samza.message.WatermarkMessage;
-import org.apache.samza.message.IntermediateMessageType;
 import org.apache.samza.serializers.IntermediateMessageSerde;
 import org.apache.samza.serializers.Serde;
+import org.apache.samza.system.EndOfStreamMessage;
+import org.apache.samza.system.MessageType;
+import org.apache.samza.system.WatermarkMessage;
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
@@ -96,7 +96,7 @@ public class TestIntermediateMessageSerde {
     TestUserMessage userMessage = new TestUserMessage(msg, 0, System.currentTimeMillis());
     byte[] bytes = imserde.toBytes(userMessage);
     TestUserMessage de = (TestUserMessage) imserde.fromBytes(bytes);
-    assertEquals(IntermediateMessageType.of(de), IntermediateMessageType.USER_MESSAGE);
+    assertEquals(MessageType.of(de), MessageType.USER_MESSAGE);
     assertEquals(de.getMessage(), msg);
     assertEquals(de.getOffset(), 0);
     assertTrue(de.getTimestamp() > 0);
@@ -106,12 +106,11 @@ public class TestIntermediateMessageSerde {
   public void testWatermarkMessageSerde() {
     IntermediateMessageSerde imserde = new IntermediateMessageSerde(new ObjectSerde());
     String taskName = "task-1";
-    WatermarkMessage watermark = new WatermarkMessage(System.currentTimeMillis(), taskName, 8);
+    WatermarkMessage watermark = new WatermarkMessage(System.currentTimeMillis(), taskName);
     byte[] bytes = imserde.toBytes(watermark);
     WatermarkMessage de = (WatermarkMessage) imserde.fromBytes(bytes);
-    assertEquals(IntermediateMessageType.of(de), IntermediateMessageType.WATERMARK_MESSAGE);
+    assertEquals(MessageType.of(de), MessageType.WATERMARK);
     assertEquals(de.getTaskName(), taskName);
-    assertEquals(de.getTaskCount(), 8);
     assertTrue(de.getTimestamp() > 0);
   }
 
@@ -120,12 +119,11 @@ public class TestIntermediateMessageSerde {
     IntermediateMessageSerde imserde = new IntermediateMessageSerde(new ObjectSerde());
     String streamId = "test-stream";
     String taskName = "task-1";
-    EndOfStreamMessage eos = new EndOfStreamMessage(taskName, 8);
+    EndOfStreamMessage eos = new EndOfStreamMessage(taskName);
     byte[] bytes = imserde.toBytes(eos);
     EndOfStreamMessage de = (EndOfStreamMessage) imserde.fromBytes(bytes);
-    assertEquals(IntermediateMessageType.of(de), IntermediateMessageType.END_OF_STREAM_MESSAGE);
+    assertEquals(MessageType.of(de), MessageType.END_OF_STREAM);
     assertEquals(de.getTaskName(), taskName);
-    assertEquals(de.getTaskCount(), 8);
     assertEquals(de.getVersion(), 1);
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java b/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java
index 3d4976b..5a4b4bf 100644
--- a/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java
+++ b/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java
@@ -39,7 +39,6 @@ import org.apache.samza.container.TaskInstance;
 import org.apache.samza.container.TaskInstanceExceptionHandler;
 import org.apache.samza.container.TaskInstanceMetrics;
 import org.apache.samza.container.TaskName;
-import org.apache.samza.control.EndOfStreamManager;
 import org.apache.samza.metrics.MetricsRegistryMap;
 import org.apache.samza.system.IncomingMessageEnvelope;
 import org.apache.samza.system.SystemConsumer;
@@ -78,8 +77,8 @@ public class TestAsyncRunLoop {
   private final IncomingMessageEnvelope envelope0 = new IncomingMessageEnvelope(ssp0, "0", "key0", "value0");
   private final IncomingMessageEnvelope envelope1 = new IncomingMessageEnvelope(ssp1, "1", "key1", "value1");
   private final IncomingMessageEnvelope envelope3 = new IncomingMessageEnvelope(ssp0, "1", "key0", "value0");
-  private final IncomingMessageEnvelope ssp0EndOfStream = EndOfStreamManager.buildEndOfStreamEnvelope(ssp0);
-  private final IncomingMessageEnvelope ssp1EndOfStream = EndOfStreamManager.buildEndOfStreamEnvelope(ssp1);
+  private final IncomingMessageEnvelope ssp0EndOfStream = IncomingMessageEnvelope.buildEndOfStreamEnvelope(ssp0);
+  private final IncomingMessageEnvelope ssp1EndOfStream = IncomingMessageEnvelope.buildEndOfStreamEnvelope(ssp1);
 
   TaskInstance createTaskInstance(AsyncStreamTask task, TaskName taskName, SystemStreamPartition ssp, OffsetManager manager, SystemConsumers consumers) {
     TaskInstanceMetrics taskInstanceMetrics = new TaskInstanceMetrics("task", new MetricsRegistryMap());
@@ -575,7 +574,7 @@ public class TestAsyncRunLoop {
     SystemStreamPartition ssp2 = new SystemStreamPartition("system1", "stream2", p2);
     IncomingMessageEnvelope envelope1 = new IncomingMessageEnvelope(ssp2, "1", "key1", "message1");
     IncomingMessageEnvelope envelope2 = new IncomingMessageEnvelope(ssp2, "2", "key1", "message1");
-    IncomingMessageEnvelope envelope3 = EndOfStreamManager.buildEndOfStreamEnvelope(ssp2);
+    IncomingMessageEnvelope envelope3 = IncomingMessageEnvelope.buildEndOfStreamEnvelope(ssp2);
 
     Map<SystemStreamPartition, List<IncomingMessageEnvelope>> sspMap = new HashMap<>();
     List<IncomingMessageEnvelope> messageList = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala b/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
index 81f3ed1..ea11d9f 100644
--- a/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
+++ b/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
@@ -20,28 +20,21 @@
 package org.apache.samza.container
 
 
-import java.util
-import java.util
-import java.util.Collections
 import java.util.concurrent.ConcurrentHashMap
-import com.google.common.collect.Multimap
-import org.apache.samza.SamzaException
 
 import org.apache.samza.Partition
-import org.apache.samza.checkpoint.OffsetManager
-import org.apache.samza.config.Config
-import org.apache.samza.config.MapConfig
-import org.apache.samza.control.ControlMessageUtils
-import org.apache.samza.job.model.ContainerModel
-import org.apache.samza.job.model.JobModel
-import org.apache.samza.job.model.TaskModel
-import org.apache.samza.metrics.Counter
-import org.apache.samza.metrics.Metric
-import org.apache.samza.metrics.MetricsRegistryMap
 import org.apache.samza.checkpoint.{Checkpoint, OffsetManager}
 import org.apache.samza.config.{Config, MapConfig}
 import org.apache.samza.metrics.{Counter, Metric, MetricsRegistryMap}
 import org.apache.samza.serializers.SerdeManager
+import org.apache.samza.system.IncomingMessageEnvelope
+import org.apache.samza.system.SystemAdmin
+import org.apache.samza.system.SystemConsumer
+import org.apache.samza.system.SystemConsumers
+import org.apache.samza.system.SystemProducer
+import org.apache.samza.system.SystemProducers
+import org.apache.samza.system.SystemStream
+import org.apache.samza.system.SystemStreamMetadata
 import org.apache.samza.storage.TaskStorageManager
 import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata
 import org.apache.samza.system._
@@ -124,9 +117,9 @@ class TestTaskInstance {
    */
   class TroublesomeTask extends StreamTask with WindowableTask {
     def process(
-      envelope: IncomingMessageEnvelope,
-      collector: MessageCollector,
-      coordinator: TaskCoordinator) {
+                 envelope: IncomingMessageEnvelope,
+                 collector: MessageCollector,
+                 coordinator: TaskCoordinator) {
 
       envelope.getOffset().toInt match {
         case offset if offset % 2 == 0 => throw new TroublesomeException
@@ -143,8 +136,8 @@ class TestTaskInstance {
    * Helper method used to retrieve the value of a counter from a group.
    */
   private def getCount(
-    group: ConcurrentHashMap[String, Metric],
-    name: String): Long = {
+                        group: ConcurrentHashMap[String, Metric],
+                        name: String): Long = {
     group.get("exception-ignored-" + name.toLowerCase).asInstanceOf[Counter].getCount
   }
 
@@ -407,36 +400,6 @@ class TestTaskInstance {
     // Finally, checkpoint the inputs with the snapshotted checkpoint captured at the beginning of commit
     mockOrder.verify(offsetManager).writeCheckpoint(taskName, checkpoint)
   }
-
-  @Test
-  def testBuildInputToTasks = {
-    val system: String = "test-system"
-    val stream0: String = "test-stream-0"
-    val stream1: String = "test-stream-1"
-
-    val ssp0: SystemStreamPartition = new SystemStreamPartition(system, stream0, new Partition(0))
-    val ssp1: SystemStreamPartition = new SystemStreamPartition(system, stream0, new Partition(1))
-    val ssp2: SystemStreamPartition = new SystemStreamPartition(system, stream1, new Partition(0))
-
-    val task0: TaskName = new TaskName("Task 0")
-    val task1: TaskName = new TaskName("Task 1")
-    val ssps: util.Set[SystemStreamPartition] = new util.HashSet[SystemStreamPartition]
-    ssps.add(ssp0)
-    ssps.add(ssp2)
-    val tm0: TaskModel = new TaskModel(task0, ssps, new Partition(0))
-    val cm0: ContainerModel = new ContainerModel("c0", 0, Collections.singletonMap(task0, tm0))
-    val tm1: TaskModel = new TaskModel(task1, Collections.singleton(ssp1), new Partition(1))
-    val cm1: ContainerModel = new ContainerModel("c1", 1, Collections.singletonMap(task1, tm1))
-
-    val cms: util.Map[String, ContainerModel] = new util.HashMap[String, ContainerModel]
-    cms.put(cm0.getProcessorId, cm0)
-    cms.put(cm1.getProcessorId, cm1)
-
-    val jobModel: JobModel = new JobModel(new MapConfig, cms, null)
-    val streamToTasks: Multimap[SystemStream, String] = TaskInstance.buildInputToTasks(jobModel)
-    assertEquals(streamToTasks.get(ssp0.getSystemStream).size, 2)
-    assertEquals(streamToTasks.get(ssp2.getSystemStream).size, 1)
-  }
 }
 
 class MockSystemAdmin extends SystemAdmin {

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/test/scala/org/apache/samza/serializers/TestSerdeManager.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/serializers/TestSerdeManager.scala b/samza-core/src/test/scala/org/apache/samza/serializers/TestSerdeManager.scala
index 9d808cb..774230c 100644
--- a/samza-core/src/test/scala/org/apache/samza/serializers/TestSerdeManager.scala
+++ b/samza-core/src/test/scala/org/apache/samza/serializers/TestSerdeManager.scala
@@ -20,8 +20,8 @@
 package org.apache.samza.serializers
 
 
-import org.apache.samza.message.EndOfStreamMessage
-import org.apache.samza.message.WatermarkMessage
+import org.apache.samza.system.EndOfStreamMessage
+import org.apache.samza.system.WatermarkMessage
 import org.junit.Assert._
 import org.junit.Assert.assertEquals
 import org.junit.Assert.assertEquals
@@ -83,18 +83,17 @@ class TestSerdeManager {
     val eosStreamId = "eos-stream"
     val taskName = "task 1"
     val taskCount = 8
-    outEnvelope = new OutgoingMessageEnvelope(intermediate, "eos", new EndOfStreamMessage(taskName, taskCount))
+    outEnvelope = new OutgoingMessageEnvelope(intermediate, "eos", new EndOfStreamMessage(taskName))
     se = serdeManager.toBytes(outEnvelope)
     inEnvelope = new IncomingMessageEnvelope(new SystemStreamPartition(intermediate, new Partition(0)), "offset", se.getKey, se.getMessage)
     de = serdeManager.fromBytes(inEnvelope)
     assertEquals(de.getKey, "eos")
     val eosMsg = de.getMessage.asInstanceOf[EndOfStreamMessage]
     assertEquals(eosMsg.getTaskName, taskName)
-    assertEquals(eosMsg.getTaskCount, taskCount)
 
     // test watermark message sent to intermediate stream
     val timestamp = System.currentTimeMillis()
-    outEnvelope = new OutgoingMessageEnvelope(intermediate, "watermark", new WatermarkMessage(timestamp, taskName, taskCount))
+    outEnvelope = new OutgoingMessageEnvelope(intermediate, "watermark", new WatermarkMessage(timestamp, taskName))
     se = serdeManager.toBytes(outEnvelope)
     inEnvelope = new IncomingMessageEnvelope(new SystemStreamPartition(intermediate, new Partition(0)), "offset", se.getKey, se.getMessage)
     de = serdeManager.fromBytes(inEnvelope)
@@ -102,6 +101,5 @@ class TestSerdeManager {
     val watermarkMsg = de.getMessage.asInstanceOf[WatermarkMessage]
     assertEquals(watermarkMsg.getTimestamp, timestamp)
     assertEquals(watermarkMsg.getTaskName, taskName)
-    assertEquals(watermarkMsg.getTaskCount, taskCount)
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java
----------------------------------------------------------------------
diff --git a/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java b/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java
index de0d1da..fb9bb56 100644
--- a/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java
+++ b/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java
@@ -36,7 +36,6 @@ import org.apache.commons.lang.Validate;
 import org.apache.samza.Partition;
 import org.apache.samza.SamzaException;
 import org.apache.samza.config.Config;
-import org.apache.samza.control.EndOfStreamManager;
 import org.apache.samza.metrics.Counter;
 import org.apache.samza.metrics.MetricsRegistry;
 import org.apache.samza.system.IncomingMessageEnvelope;
@@ -237,7 +236,7 @@ public class HdfsSystemConsumer extends BlockingEnvelopeMap {
       consumerMetrics.incNumEvents(systemStreamPartition);
       consumerMetrics.incTotalNumEvents();
     }
-    offerMessage(systemStreamPartition, EndOfStreamManager.buildEndOfStreamEnvelope(systemStreamPartition));
+    offerMessage(systemStreamPartition, IncomingMessageEnvelope.buildEndOfStreamEnvelope(systemStreamPartition));
     reader.close();
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
index f313348..8493cf1 100644
--- a/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
+++ b/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
@@ -54,7 +54,7 @@ public class EndOfStreamIntegrationTest extends AbstractIntegrationTestHarness {
   @Test
   public void testPipeline() throws  Exception {
     Random random = new Random();
-    int count = 100;
+    int count = 10;
     PageView[] pageviews = new PageView[count];
     for (int i = 0; i < count; i++) {
       String pagekey = PAGEKEYS[random.nextInt(PAGEKEYS.length - 1)];

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java
index 2eb72fc..d9202d3 100644
--- a/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java
+++ b/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java
@@ -36,8 +36,6 @@ import org.apache.samza.container.SamzaContainer;
 import org.apache.samza.container.TaskInstance;
 import org.apache.samza.container.TaskName;
 import org.apache.samza.container.grouper.task.SingleContainerGrouperFactory;
-import org.apache.samza.control.EndOfStreamManager;
-import org.apache.samza.control.WatermarkManager;
 import org.apache.samza.metrics.MetricsRegistry;
 import org.apache.samza.operators.impl.InputOperatorImpl;
 import org.apache.samza.operators.impl.OperatorImpl;
@@ -85,14 +83,14 @@ public class WatermarkIntegrationTest extends AbstractIntegrationTestHarness {
   static {
     TEST_DATA.add(createIncomingMessage(new PageView("inbox", 1), SSP0));
     TEST_DATA.add(createIncomingMessage(new PageView("home", 2), SSP1));
-    TEST_DATA.add(WatermarkManager.buildWatermarkEnvelope(1, SSP0));
-    TEST_DATA.add(WatermarkManager.buildWatermarkEnvelope(2, SSP1));
-    TEST_DATA.add(WatermarkManager.buildWatermarkEnvelope(4, SSP0));
-    TEST_DATA.add(WatermarkManager.buildWatermarkEnvelope(3, SSP1));
+    TEST_DATA.add(IncomingMessageEnvelope.buildWatermarkEnvelope(SSP0, 1));
+    TEST_DATA.add(IncomingMessageEnvelope.buildWatermarkEnvelope(SSP1, 2));
+    TEST_DATA.add(IncomingMessageEnvelope.buildWatermarkEnvelope(SSP0, 4));
+    TEST_DATA.add(IncomingMessageEnvelope.buildWatermarkEnvelope(SSP1, 3));
     TEST_DATA.add(createIncomingMessage(new PageView("search", 3), SSP0));
     TEST_DATA.add(createIncomingMessage(new PageView("pymk", 4), SSP1));
-    TEST_DATA.add(EndOfStreamManager.buildEndOfStreamEnvelope(SSP0));
-    TEST_DATA.add(EndOfStreamManager.buildEndOfStreamEnvelope(SSP1));
+    TEST_DATA.add(IncomingMessageEnvelope.buildEndOfStreamEnvelope(SSP0));
+    TEST_DATA.add(IncomingMessageEnvelope.buildEndOfStreamEnvelope(SSP1));
   }
 
   public final static class TestSystemFactory implements SystemFactory {

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-test/src/test/java/org/apache/samza/test/util/ArraySystemConsumer.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/util/ArraySystemConsumer.java b/samza-test/src/test/java/org/apache/samza/test/util/ArraySystemConsumer.java
index 9b96216..832457b 100644
--- a/samza-test/src/test/java/org/apache/samza/test/util/ArraySystemConsumer.java
+++ b/samza-test/src/test/java/org/apache/samza/test/util/ArraySystemConsumer.java
@@ -27,7 +27,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
 import org.apache.samza.config.Config;
-import org.apache.samza.control.EndOfStreamManager;
 import org.apache.samza.system.IncomingMessageEnvelope;
 import org.apache.samza.system.SystemConsumer;
 import org.apache.samza.system.SystemStreamPartition;
@@ -62,7 +61,7 @@ public class ArraySystemConsumer implements SystemConsumer {
       set.forEach(ssp -> {
           List<IncomingMessageEnvelope> envelopes = Arrays.stream(getArrayObjects(ssp.getSystemStream().getStream(), config))
               .map(object -> new IncomingMessageEnvelope(ssp, null, null, object)).collect(Collectors.toList());
-          envelopes.add(EndOfStreamManager.buildEndOfStreamEnvelope(ssp));
+          envelopes.add(IncomingMessageEnvelope.buildEndOfStreamEnvelope(ssp));
           envelopeMap.put(ssp, envelopes);
         });
       done = true;


[04/16] samza git commit: SAMZA-1321: Propagate end-of-stream and watermark messages

Posted by xi...@apache.org.
SAMZA-1321: Propagate end-of-stream and watermark messages

The patch completes the end-of-stream work flow across multi-stage pipeline. It also contains initial commit for supporting watermarks. For watermark, there are issues raised in the review feedback and will be addressed by further prs. The main logic this patch adds:

- EndOfStreamManager aggregates the end-of-stream control messages, propagate the result to to downstream intermediate topics based on the topology of the IO in the StreamGraph.

- WatermarkManager aggregates the watermark control messages from the upstage tasks, pass it through the operators, and propagate it to downstream.

In operator impl, I implemented similar watermark logic as Beam for watermark propagation:
* InputWatermark(op) = min { OutputWatermark(op') | op1 is upstream of op}
* OutputWatermark(op) = min { InputWatermark(op), OldestWorkTime(op) }

Add quite a few unit tests and integration test. The code is 100% covered as reported by Intellij. Both control messages work as expected.

Author: Xinyu Liu <xi...@xiliu-ld.linkedin.biz>

Reviewers: Yi Pan <ni...@gmail.com>

Closes #236 from xinyuiscool/SAMZA-1321


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

Branch: refs/heads/master
Commit: bb3007d69ae177791aeb00ba5b17fd937628b2b8
Parents: 6815a4d
Author: Xinyu Liu <xi...@xiliu-ld.linkedin.biz>
Authored: Wed Jun 28 17:16:10 2017 -0700
Committer: Xinyu Liu <xi...@xiliu-ld.linkedin.biz>
Committed: Wed Jun 28 17:16:10 2017 -0700

----------------------------------------------------------------------
 .../samza/system/IncomingMessageEnvelope.java   |   4 +-
 .../org/apache/samza/system/StreamSpec.java     |   4 +
 .../control/ControlMessageListenerTask.java     |  49 +++
 .../samza/control/ControlMessageUtils.java      |  81 +++++
 .../samza/control/EndOfStreamManager.java       | 159 +++++++++
 .../java/org/apache/samza/control/IOGraph.java  | 113 +++++++
 .../org/apache/samza/control/Watermark.java     |  57 ++++
 .../apache/samza/control/WatermarkManager.java  | 187 +++++++++++
 .../samza/message/IntermediateMessageType.java  |  46 +++
 .../apache/samza/operators/StreamGraphImpl.java |   5 +
 .../samza/operators/impl/OperatorImpl.java      |  93 +++++-
 .../operators/impl/OutputOperatorImpl.java      |  16 +-
 .../apache/samza/processor/StreamProcessor.java |  16 +-
 .../samza/runtime/LocalApplicationRunner.java   |   5 +
 .../samza/runtime/LocalContainerRunner.java     |   6 +-
 .../serializers/IntermediateMessageSerde.java   |  14 +-
 .../org/apache/samza/task/AsyncRunLoop.java     |  12 +-
 .../samza/task/AsyncStreamTaskAdapter.java      |  22 +-
 .../apache/samza/task/StreamOperatorTask.java   |  28 +-
 .../apache/samza/container/SamzaContainer.scala |  13 +-
 .../apache/samza/container/TaskInstance.scala   | 141 ++++++--
 .../samza/job/local/ThreadJobFactory.scala      |   6 +-
 .../samza/system/StreamMetadataCache.scala      |  11 +
 .../samza/control/TestControlMessageUtils.java  | 115 +++++++
 .../samza/control/TestEndOfStreamManager.java   | 333 +++++++++++++++++++
 .../org/apache/samza/control/TestIOGraph.java   | 200 +++++++++++
 .../samza/control/TestWatermarkManager.java     | 260 +++++++++++++++
 .../samza/operators/impl/TestOperatorImpl.java  |  18 +
 .../samza/processor/TestStreamProcessor.java    |   4 +-
 .../runtime/TestLocalApplicationRunner.java     |   6 +-
 .../TestIntermediateMessageSerde.java           |   8 +-
 .../org/apache/samza/task/TestAsyncRunLoop.java |   9 +-
 .../samza/task/TestStreamOperatorTask.java      |  30 ++
 .../samza/container/TestTaskInstance.scala      |  39 +++
 .../samza/system/hdfs/HdfsSystemConsumer.java   |   3 +-
 .../processor/TestStreamProcessorUtil.java      |  28 ++
 .../EndOfStreamIntegrationTest.java             | 102 ++++++
 .../samza/test/controlmessages/TestData.java    |  85 +++++
 .../WatermarkIntegrationTest.java               | 204 ++++++++++++
 .../samza/test/util/ArraySystemConsumer.java    |  83 +++++
 .../samza/test/util/ArraySystemFactory.java     |  50 +++
 .../samza/test/util/Base64Serializer.java       |  66 ++++
 .../samza/test/util/SimpleSystemAdmin.java      |  90 +++++
 .../samza/test/util/TestStreamConsumer.java     |  52 +++
 44 files changed, 2792 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-api/src/main/java/org/apache/samza/system/IncomingMessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/IncomingMessageEnvelope.java b/samza-api/src/main/java/org/apache/samza/system/IncomingMessageEnvelope.java
index 0ced773..9182522 100644
--- a/samza-api/src/main/java/org/apache/samza/system/IncomingMessageEnvelope.java
+++ b/samza-api/src/main/java/org/apache/samza/system/IncomingMessageEnvelope.java
@@ -91,12 +91,12 @@ public class IncomingMessageEnvelope {
   }
 
   /**
-   * Builds an end-of-stream envelope for an SSP. This is used by a {@link SystemConsumer} implementation to
-   * indicate that it is at end-of-stream. The end-of-stream message should not delivered to the task implementation.
+   * This method is deprecated in favor of WatermarkManager.buildEndOfStreamEnvelope(SystemStreamPartition ssp).
    *
    * @param ssp The SSP that is at end-of-stream.
    * @return an IncomingMessageEnvelope corresponding to end-of-stream for that SSP.
    */
+  @Deprecated
   public static IncomingMessageEnvelope buildEndOfStreamEnvelope(SystemStreamPartition ssp) {
     return new IncomingMessageEnvelope(ssp, END_OF_STREAM_OFFSET, null, null);
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java b/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
index 0cdeb95..49531dd 100644
--- a/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
+++ b/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
@@ -196,6 +196,10 @@ public class StreamSpec {
     return config.getOrDefault(propertyName, defaultValue);
   }
 
+  public SystemStream toSystemStream() {
+    return new SystemStream(systemName, physicalName);
+  }
+
   private void validateLogicalIdentifier(String identifierName, String identifierValue) {
     if (identifierValue == null || !identifierValue.matches("[A-Za-z0-9_-]+")) {
       throw new IllegalArgumentException(String.format("Identifier '%s' is '%s'. It must match the expression [A-Za-z0-9_-]+", identifierName, identifierValue));

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/main/java/org/apache/samza/control/ControlMessageListenerTask.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/control/ControlMessageListenerTask.java b/samza-core/src/main/java/org/apache/samza/control/ControlMessageListenerTask.java
new file mode 100644
index 0000000..9e4b40a
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/control/ControlMessageListenerTask.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.control;
+
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+
+
+/**
+ * The listener interface for the aggregation result of control messages.
+ * Any task that handles control messages such as {@link org.apache.samza.message.EndOfStreamMessage}
+ * and {@link org.apache.samza.message.WatermarkMessage} needs to implement this interface.
+ */
+public interface ControlMessageListenerTask {
+
+  /**
+   * Returns the topology of the streams. Any control message listener needs to
+   * provide this topology so Samza can propagate the control message to downstreams.
+   * @return {@link IOGraph} of input to output streams. It
+   */
+  IOGraph getIOGraph();
+
+  /**
+   * Invoked when a Watermark comes.
+   * @param watermark contains the watermark timestamp
+   * @param systemStream source of stream that emits the watermark
+   * @param collector message collector
+   * @param coordinator task coordinator
+   */
+  void onWatermark(Watermark watermark, SystemStream systemStream, MessageCollector collector, TaskCoordinator coordinator);
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/main/java/org/apache/samza/control/ControlMessageUtils.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/control/ControlMessageUtils.java b/samza-core/src/main/java/org/apache/samza/control/ControlMessageUtils.java
new file mode 100644
index 0000000..ebb0d86
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/control/ControlMessageUtils.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.control;
+
+import com.google.common.collect.Multimap;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.samza.message.ControlMessage;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.StreamMetadataCache;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamMetadata;
+import org.apache.samza.task.MessageCollector;
+
+
+/**
+ * This class privates static utils for handling control messages
+ */
+class ControlMessageUtils {
+
+  /**
+   * Send a control message to every partition of the {@link SystemStream}
+   * @param message control message
+   * @param systemStream the stream to sent
+   * @param metadataCache stream metadata cache
+   * @param collector collector to send the message
+   */
+  static void sendControlMessage(ControlMessage message,
+      SystemStream systemStream,
+      StreamMetadataCache metadataCache,
+      MessageCollector collector) {
+    SystemStreamMetadata metadata = metadataCache.getSystemStreamMetadata(systemStream, true);
+    int partitionCount = metadata.getSystemStreamPartitionMetadata().size();
+    for (int i = 0; i < partitionCount; i++) {
+      OutgoingMessageEnvelope envelopeOut = new OutgoingMessageEnvelope(systemStream, i, "", message);
+      collector.send(envelopeOut);
+    }
+  }
+
+  /**
+   * Calculate the mapping from an output stream to the number of upstream tasks that will produce to the output stream
+   * @param inputToTasks input stream to its consumer tasks mapping
+   * @param ioGraph topology of the stream inputs and outputs
+   * @return mapping from output to upstream task count
+   */
+  static Map<SystemStream, Integer> calculateUpstreamTaskCounts(Multimap<SystemStream, String> inputToTasks,
+      IOGraph ioGraph) {
+    if (ioGraph == null) {
+      return Collections.EMPTY_MAP;
+    }
+    Map<SystemStream, Integer> outputTaskCount = new HashMap<>();
+    ioGraph.getNodes().forEach(node -> {
+        // for each input stream, find out the tasks that are consuming this input stream using the inputToTasks mapping,
+        // then count the number of tasks
+        int count = node.getInputs().stream().flatMap(spec -> inputToTasks.get(spec.toSystemStream()).stream())
+            .collect(Collectors.toSet()).size();
+        // put the count of input tasks to the result
+        outputTaskCount.put(node.getOutput().toSystemStream(), count);
+      });
+    return outputTaskCount;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/main/java/org/apache/samza/control/EndOfStreamManager.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/control/EndOfStreamManager.java b/samza-core/src/main/java/org/apache/samza/control/EndOfStreamManager.java
new file mode 100644
index 0000000..78a8741
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/control/EndOfStreamManager.java
@@ -0,0 +1,159 @@
+/*
+ * 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.control;
+
+import com.google.common.collect.Multimap;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import org.apache.samza.message.EndOfStreamMessage;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.StreamMetadataCache;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This class handles the end-of-stream control message. It aggregates the end-of-stream state for each input ssps of
+ * a task, and propagate the eos messages to downstream intermediate streams if needed.
+ *
+ * Internal use only.
+ */
+public class EndOfStreamManager {
+  private static final Logger log = LoggerFactory.getLogger(EndOfStreamManager.class);
+
+  private final String taskName;
+  private final MessageCollector collector;
+  // end-of-stream state per ssp
+  private final Map<SystemStreamPartition, EndOfStreamState> eosStates;
+  private final StreamMetadataCache metadataCache;
+  // topology information. Set during init()
+  private final ControlMessageListenerTask listener;
+  // mapping from output stream to its upstream task count
+  private final Map<SystemStream, Integer> upstreamTaskCounts;
+
+  public EndOfStreamManager(String taskName,
+      ControlMessageListenerTask listener,
+      Multimap<SystemStream, String> inputToTasks,
+      Set<SystemStreamPartition> ssps,
+      StreamMetadataCache metadataCache,
+      MessageCollector collector) {
+    this.taskName = taskName;
+    this.listener = listener;
+    this.metadataCache = metadataCache;
+    this.collector = collector;
+    Map<SystemStreamPartition, EndOfStreamState> states = new HashMap<>();
+    ssps.forEach(ssp -> {
+        states.put(ssp, new EndOfStreamState());
+      });
+    this.eosStates = Collections.unmodifiableMap(states);
+    this.upstreamTaskCounts = ControlMessageUtils.calculateUpstreamTaskCounts(inputToTasks, listener.getIOGraph());
+  }
+
+  public void update(IncomingMessageEnvelope envelope, TaskCoordinator coordinator) {
+    EndOfStreamState state = eosStates.get(envelope.getSystemStreamPartition());
+    EndOfStreamMessage message = (EndOfStreamMessage) envelope.getMessage();
+    state.update(message.getTaskName(), message.getTaskCount());
+    log.info("Received end-of-stream from task " + message.getTaskName() + " in " + envelope.getSystemStreamPartition());
+
+    SystemStream systemStream = envelope.getSystemStreamPartition().getSystemStream();
+    if (isEndOfStream(systemStream)) {
+      log.info("End-of-stream of input " + systemStream + " for " + systemStream);
+      listener.getIOGraph().getNodesOfInput(systemStream).forEach(node -> {
+          // find the intermediate streams that need broadcast the eos messages
+          if (node.isOutputIntermediate()) {
+            // check all the input stream partitions assigned to the task are end-of-stream
+            boolean inputsEndOfStream = node.getInputs().stream().allMatch(spec -> isEndOfStream(spec.toSystemStream()));
+            if (inputsEndOfStream) {
+              // broadcast the end-of-stream message to the intermediate stream
+              SystemStream outputStream = node.getOutput().toSystemStream();
+              sendEndOfStream(outputStream, upstreamTaskCounts.get(outputStream));
+            }
+          }
+        });
+
+      boolean allEndOfStream = eosStates.values().stream().allMatch(EndOfStreamState::isEndOfStream);
+      if (allEndOfStream) {
+        // all inputs have been end-of-stream, shut down the task
+        log.info("All input streams have reached the end for task " + taskName);
+        coordinator.shutdown(TaskCoordinator.RequestScope.CURRENT_TASK);
+      }
+    }
+  }
+
+  /**
+   * Return true if all partitions of the systemStream that are assigned to the current task have reached EndOfStream.
+   * @param systemStream stream
+   * @return whether the stream reaches to the end for this task
+   */
+  boolean isEndOfStream(SystemStream systemStream) {
+    return eosStates.entrySet().stream()
+        .filter(entry -> entry.getKey().getSystemStream().equals(systemStream))
+        .allMatch(entry -> entry.getValue().isEndOfStream());
+  }
+
+  /**
+   * Send the EndOfStream control messages to downstream
+   * @param systemStream downstream stream
+   */
+  void sendEndOfStream(SystemStream systemStream, int taskCount) {
+    log.info("Send end-of-stream messages with upstream task count {} to all partitions of {}", taskCount, systemStream);
+    final EndOfStreamMessage message = new EndOfStreamMessage(taskName, taskCount);
+    ControlMessageUtils.sendControlMessage(message, systemStream, metadataCache, collector);
+  }
+
+  /**
+   * This class keeps the internal state for a ssp to be end-of-stream.
+   */
+  final static class EndOfStreamState {
+    // set of upstream tasks
+    private final Set<String> tasks = new HashSet<>();
+    private int expectedTotal = Integer.MAX_VALUE;
+    private boolean isEndOfStream = false;
+
+    void update(String taskName, int taskCount) {
+      if (taskName != null) {
+        tasks.add(taskName);
+      }
+      expectedTotal = taskCount;
+      isEndOfStream = tasks.size() == expectedTotal;
+    }
+
+    boolean isEndOfStream() {
+      return isEndOfStream;
+    }
+  }
+
+  /**
+   * Build an end-of-stream envelope for an ssp of a source input.
+   *
+   * @param ssp The SSP that is at end-of-stream.
+   * @return an IncomingMessageEnvelope corresponding to end-of-stream for that SSP.
+   */
+  public static IncomingMessageEnvelope buildEndOfStreamEnvelope(SystemStreamPartition ssp) {
+    return new IncomingMessageEnvelope(ssp, IncomingMessageEnvelope.END_OF_STREAM_OFFSET, null, new EndOfStreamMessage(null, 0));
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/main/java/org/apache/samza/control/IOGraph.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/control/IOGraph.java b/samza-core/src/main/java/org/apache/samza/control/IOGraph.java
new file mode 100644
index 0000000..a30c13d
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/control/IOGraph.java
@@ -0,0 +1,113 @@
+/*
+ * 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.control;
+
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import org.apache.samza.operators.StreamGraphImpl;
+import org.apache.samza.operators.spec.OperatorSpec;
+import org.apache.samza.operators.spec.OutputOperatorSpec;
+import org.apache.samza.system.StreamSpec;
+import org.apache.samza.system.SystemStream;
+
+
+/**
+ * This class provides the topology of stream inputs to outputs.
+ */
+public class IOGraph {
+
+  public static final class IONode {
+    private final Set<StreamSpec> inputs = new HashSet<>();
+    private final StreamSpec output;
+    private final boolean isOutputIntermediate;
+
+    IONode(StreamSpec output, boolean isOutputIntermediate) {
+      this.output = output;
+      this.isOutputIntermediate = isOutputIntermediate;
+    }
+
+    void addInput(StreamSpec input) {
+      inputs.add(input);
+    }
+
+    public Set<StreamSpec> getInputs() {
+      return Collections.unmodifiableSet(inputs);
+    }
+
+    public StreamSpec getOutput() {
+      return output;
+    }
+
+    public boolean isOutputIntermediate() {
+      return isOutputIntermediate;
+    }
+  }
+
+  final Collection<IONode> nodes;
+  final Multimap<SystemStream, IONode> inputToNodes;
+
+  public IOGraph(Collection<IONode> nodes) {
+    this.nodes = Collections.unmodifiableCollection(nodes);
+    this.inputToNodes = HashMultimap.create();
+    nodes.forEach(node -> {
+        node.getInputs().forEach(stream -> {
+            inputToNodes.put(new SystemStream(stream.getSystemName(), stream.getPhysicalName()), node);
+          });
+      });
+  }
+
+  public Collection<IONode> getNodes() {
+    return this.nodes;
+  }
+
+  public Collection<IONode> getNodesOfInput(SystemStream input) {
+    return inputToNodes.get(input);
+  }
+
+  public static IOGraph buildIOGraph(StreamGraphImpl streamGraph) {
+    Map<Integer, IONode> nodes = new HashMap<>();
+    streamGraph.getInputOperators().entrySet().stream()
+        .forEach(entry -> buildIONodes(entry.getKey(), entry.getValue(), nodes));
+    return new IOGraph(nodes.values());
+  }
+
+  /* package private */
+  static void buildIONodes(StreamSpec input, OperatorSpec opSpec, Map<Integer, IONode> ioGraph) {
+    if (opSpec instanceof OutputOperatorSpec) {
+      OutputOperatorSpec outputOpSpec = (OutputOperatorSpec) opSpec;
+      IONode node = ioGraph.get(opSpec.getOpId());
+      if (node == null) {
+        StreamSpec output = outputOpSpec.getOutputStream().getStreamSpec();
+        node = new IONode(output, outputOpSpec.getOpCode() == OperatorSpec.OpCode.PARTITION_BY);
+        ioGraph.put(opSpec.getOpId(), node);
+      }
+      node.addInput(input);
+    }
+
+    Collection<OperatorSpec> nextOperators = opSpec.getRegisteredOperatorSpecs();
+    nextOperators.forEach(spec -> buildIONodes(input, spec, ioGraph));
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/main/java/org/apache/samza/control/Watermark.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/control/Watermark.java b/samza-core/src/main/java/org/apache/samza/control/Watermark.java
new file mode 100644
index 0000000..a11e3b0
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/control/Watermark.java
@@ -0,0 +1,57 @@
+/*
+ * 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.control;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.system.SystemStream;
+
+
+/**
+ * A watermark is a monotonically increasing value, which represents the point up to which the
+ * system believes it has received all of the data before the watermark timestamp. Data that arrives
+ * with a timestamp that is before the watermark is considered late.
+ *
+ * <p>This is the aggregate result from the WatermarkManager, which keeps track of the control message
+ * {@link org.apache.samza.message.WatermarkMessage} and aggregate by returning the min of all watermark timestamp
+ * in each partition.
+ */
+@InterfaceStability.Unstable
+public interface Watermark {
+  /**
+   * Returns the timestamp of the watermark
+   * Note that if the task consumes more than one partitions of this stream, the watermark emitted is the min of
+   * watermarks across all partitions.
+   * @return timestamp
+   */
+  long getTimestamp();
+
+  /**
+   * Propagates the watermark to an intermediate stream
+   * @param systemStream intermediate stream
+   */
+  void propagate(SystemStream systemStream);
+
+  /**
+   * Create a copy of the watermark with the timestamp
+   * @param timestamp new timestamp
+   * @return new watermark
+   */
+  Watermark copyWithTimestamp(long timestamp);
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/main/java/org/apache/samza/control/WatermarkManager.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/control/WatermarkManager.java b/samza-core/src/main/java/org/apache/samza/control/WatermarkManager.java
new file mode 100644
index 0000000..c4fdd88
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/control/WatermarkManager.java
@@ -0,0 +1,187 @@
+/*
+ * 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.control;
+
+import com.google.common.collect.Multimap;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import org.apache.samza.message.WatermarkMessage;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.StreamMetadataCache;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.task.MessageCollector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This class manages watermarks. It aggregates the watermark control messages from the upstage tasks
+ * for each SSP into an envelope of {@link Watermark}, and provide a dispatcher to propagate it to downstream.
+ *
+ * Internal use only.
+ */
+public class WatermarkManager {
+  private static final Logger log = LoggerFactory.getLogger(WatermarkManager.class);
+  public static final long TIME_NOT_EXIST = -1;
+
+  private final String taskName;
+  private final Map<SystemStreamPartition, WatermarkState> watermarkStates;
+  private final Map<SystemStream, Long> watermarkPerStream;
+  private final StreamMetadataCache metadataCache;
+  private final MessageCollector collector;
+  // mapping from output stream to its upstream task count
+  private final Map<SystemStream, Integer> upstreamTaskCounts;
+
+  public WatermarkManager(String taskName,
+      ControlMessageListenerTask listener,
+      Multimap<SystemStream, String> inputToTasks,
+      Set<SystemStreamPartition> ssps,
+      StreamMetadataCache metadataCache,
+      MessageCollector collector) {
+    this.taskName = taskName;
+    this.watermarkPerStream = new HashMap<>();
+    this.metadataCache = metadataCache;
+    this.collector = collector;
+    this.upstreamTaskCounts = ControlMessageUtils.calculateUpstreamTaskCounts(inputToTasks, listener.getIOGraph());
+
+    Map<SystemStreamPartition, WatermarkState> states = new HashMap<>();
+    ssps.forEach(ssp -> {
+        states.put(ssp, new WatermarkState());
+        watermarkPerStream.put(ssp.getSystemStream(), TIME_NOT_EXIST);
+      });
+    this.watermarkStates = Collections.unmodifiableMap(states);
+  }
+
+  /**
+   * Update the watermark based on the incoming watermark message. The message contains
+   * a timestamp and the upstream producer task. The aggregation result is the minimal value
+   * of all watermarks for the stream:
+   * <ul>
+   *   <li>Watermark(ssp) = min { Watermark(task) | task is upstream producer and the count equals total expected tasks } </li>
+   *   <li>Watermark(stream) = min { Watermark(ssp) | ssp is a partition of stream that assigns to this task } </li>
+   * </ul>
+   *
+   * @param envelope the envelope contains {@link WatermarkMessage}
+   * @return watermark envelope if there is a new aggregate watermark for the stream
+   */
+  public Watermark update(IncomingMessageEnvelope envelope) {
+    SystemStreamPartition ssp = envelope.getSystemStreamPartition();
+    WatermarkState state = watermarkStates.get(ssp);
+    WatermarkMessage message = (WatermarkMessage) envelope.getMessage();
+    state.update(message.getTimestamp(), message.getTaskName(), message.getTaskCount());
+
+    if (state.getWatermarkTime() != TIME_NOT_EXIST) {
+      long minTimestamp = watermarkStates.entrySet().stream()
+          .filter(entry -> entry.getKey().getSystemStream().equals(ssp.getSystemStream()))
+          .map(entry -> entry.getValue().getWatermarkTime())
+          .min(Long::compare)
+          .get();
+      Long curWatermark = watermarkPerStream.get(ssp.getSystemStream());
+      if (curWatermark == null || curWatermark < minTimestamp) {
+        watermarkPerStream.put(ssp.getSystemStream(), minTimestamp);
+        return new WatermarkImpl(minTimestamp);
+      }
+    }
+
+    return null;
+  }
+
+  /* package private */
+  long getWatermarkTime(SystemStreamPartition ssp) {
+    return watermarkStates.get(ssp).getWatermarkTime();
+  }
+
+  /**
+   * Send the watermark message to all partitions of an intermediate stream
+   * @param timestamp watermark timestamp
+   * @param systemStream intermediate stream
+   */
+  void sendWatermark(long timestamp, SystemStream systemStream, int taskCount) {
+    log.info("Send end-of-stream messages to all partitions of " + systemStream);
+    final WatermarkMessage watermarkMessage = new WatermarkMessage(timestamp, taskName, taskCount);
+    ControlMessageUtils.sendControlMessage(watermarkMessage, systemStream, metadataCache, collector);
+  }
+
+  /**
+   * Per ssp state of the watermarks. This class keeps track of the latest watermark timestamp
+   * from each upstream producer tasks, and use the min to update the aggregated watermark time.
+   */
+  final static class WatermarkState {
+    private int expectedTotal = Integer.MAX_VALUE;
+    private final Map<String, Long> timestamps = new HashMap<>();
+    private long watermarkTime = TIME_NOT_EXIST;
+
+    void update(long timestamp, String taskName, int taskCount) {
+      if (taskName != null) {
+        timestamps.put(taskName, timestamp);
+      }
+      expectedTotal = taskCount;
+
+      if (timestamps.size() == expectedTotal) {
+        Optional<Long> min = timestamps.values().stream().min(Long::compare);
+        watermarkTime = min.orElse(timestamp);
+      }
+    }
+
+    long getWatermarkTime() {
+      return watermarkTime;
+    }
+  }
+
+  /**
+   * Implementation of the Watermark. It keeps a reference to the {@link WatermarkManager}
+   */
+  class WatermarkImpl implements Watermark {
+    private final long timestamp;
+
+    WatermarkImpl(long timestamp) {
+      this.timestamp = timestamp;
+    }
+
+    @Override
+    public long getTimestamp() {
+      return timestamp;
+    }
+
+    @Override
+    public void propagate(SystemStream systemStream) {
+      sendWatermark(timestamp, systemStream, upstreamTaskCounts.get(systemStream));
+    }
+
+    @Override
+    public Watermark copyWithTimestamp(long time) {
+      return new WatermarkImpl(time);
+    }
+  }
+
+  /**
+   * Build a watermark control message envelope for an ssp of a source input.
+   * @param timestamp watermark time
+   * @param ssp {@link SystemStreamPartition} where the watermark coming from.
+   * @return envelope of the watermark control message
+   */
+  public static IncomingMessageEnvelope buildWatermarkEnvelope(long timestamp, SystemStreamPartition ssp) {
+    return new IncomingMessageEnvelope(ssp, null, "", new WatermarkMessage(timestamp, null, 0));
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/main/java/org/apache/samza/message/IntermediateMessageType.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/message/IntermediateMessageType.java b/samza-core/src/main/java/org/apache/samza/message/IntermediateMessageType.java
new file mode 100644
index 0000000..25fbb14
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/message/IntermediateMessageType.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.message;
+
+/**
+ * The type of the intermediate stream message. The enum will be encoded using its ordinal value and
+ * put in the first byte of the serialization of intermediate message.
+ * For more details, see {@link org.apache.samza.serializers.IntermediateMessageSerde}
+ */
+public enum IntermediateMessageType {
+  USER_MESSAGE,
+  WATERMARK_MESSAGE,
+  END_OF_STREAM_MESSAGE;
+
+  /**
+   * Returns the {@link IntermediateMessageType} of a particular intermediate stream message.
+   * @param message an intermediate stream message
+   * @return type of the message
+   */
+  public static IntermediateMessageType of(Object message) {
+    if (message instanceof WatermarkMessage) {
+      return WATERMARK_MESSAGE;
+    } else if (message instanceof EndOfStreamMessage) {
+      return END_OF_STREAM_MESSAGE;
+    } else {
+      return USER_MESSAGE;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
index c0da1b2..8718c06 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
@@ -24,6 +24,7 @@ import org.apache.samza.operators.spec.InputOperatorSpec;
 import org.apache.samza.operators.spec.OutputStreamImpl;
 import org.apache.samza.operators.stream.IntermediateMessageStreamImpl;
 import org.apache.samza.operators.spec.OperatorSpec;
+import org.apache.samza.control.IOGraph;
 import org.apache.samza.runtime.ApplicationRunner;
 import org.apache.samza.system.StreamSpec;
 
@@ -205,4 +206,8 @@ public class StreamGraphImpl implements StreamGraph {
 
     return windowOrJoinSpecs.size() != 0;
   }
+
+  public IOGraph toIOGraph() {
+    return IOGraph.buildIOGraph(this);
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
index 73bb83d..74e6748 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
@@ -18,8 +18,14 @@
  */
 package org.apache.samza.operators.impl;
 
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.MetricsConfig;
+import org.apache.samza.control.Watermark;
+import org.apache.samza.control.WatermarkManager;
 import org.apache.samza.metrics.Counter;
 import org.apache.samza.metrics.MetricsRegistry;
 import org.apache.samza.metrics.Timer;
@@ -29,11 +35,6 @@ import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
 import org.apache.samza.util.HighResolutionClock;
 
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-
 
 /**
  * Abstract base class for all stream operator implementations.
@@ -47,8 +48,11 @@ public abstract class OperatorImpl<M, RM> {
   private Counter numMessage;
   private Timer handleMessageNs;
   private Timer handleTimerNs;
+  private long inputWatermarkTime = WatermarkManager.TIME_NOT_EXIST;
+  private long outputWatermarkTime = WatermarkManager.TIME_NOT_EXIST;
 
   Set<OperatorImpl<RM, ?>> registeredOperators;
+  Set<OperatorImpl<?, M>> prevOperators;
 
   /**
    * Initialize this {@link OperatorImpl} and its user-defined functions.
@@ -69,6 +73,7 @@ public abstract class OperatorImpl<M, RM> {
 
     this.highResClock = createHighResClock(config);
     registeredOperators = new HashSet<>();
+    prevOperators = new HashSet<>();
     MetricsRegistry metricsRegistry = context.getMetricsRegistry();
     this.numMessage = metricsRegistry.newCounter(METRICS_GROUP, opName + "-messages");
     this.handleMessageNs = metricsRegistry.newTimer(METRICS_GROUP, opName + "-handle-message-ns");
@@ -99,6 +104,11 @@ public abstract class OperatorImpl<M, RM> {
               getOperatorName()));
     }
     this.registeredOperators.add(nextOperator);
+    nextOperator.registerPrevOperator(this);
+  }
+
+  void registerPrevOperator(OperatorImpl<?, M> prevOperator) {
+    this.prevOperators.add(prevOperator);
   }
 
   /**
@@ -117,9 +127,7 @@ public abstract class OperatorImpl<M, RM> {
     long endNs = this.highResClock.nanoTime();
     this.handleMessageNs.update(endNs - startNs);
 
-    results.forEach(rm ->
-        this.registeredOperators.forEach(op ->
-            op.onMessage(rm, collector, coordinator)));
+    results.forEach(rm -> this.registeredOperators.forEach(op -> op.onMessage(rm, collector, coordinator)));
   }
 
   /**
@@ -147,9 +155,7 @@ public abstract class OperatorImpl<M, RM> {
     long endNs = this.highResClock.nanoTime();
     this.handleTimerNs.update(endNs - startNs);
 
-    results.forEach(rm ->
-        this.registeredOperators.forEach(op ->
-            op.onMessage(rm, collector, coordinator)));
+    results.forEach(rm -> this.registeredOperators.forEach(op -> op.onMessage(rm, collector, coordinator)));
     this.registeredOperators.forEach(op ->
         op.onTimer(collector, coordinator));
   }
@@ -167,6 +173,71 @@ public abstract class OperatorImpl<M, RM> {
     return Collections.emptyList();
   }
 
+  /**
+   * Populate the watermarks based on the following equations:
+   *
+   * <ul>
+   *   <li>InputWatermark(op) = min { OutputWatermark(op') | op1 is upstream of op}</li>
+   *   <li>OutputWatermark(op) = min { InputWatermark(op), OldestWorkTime(op) }</li>
+   * </ul>
+   *
+   * @param watermark incoming watermark
+   * @param collector message collector
+   * @param coordinator task coordinator
+   */
+  public final void onWatermark(Watermark watermark,
+      MessageCollector collector,
+      TaskCoordinator coordinator) {
+    final long inputWatermarkMin;
+    if (prevOperators.isEmpty()) {
+      // for input operator, use the watermark time coming from the source input
+      inputWatermarkMin = watermark.getTimestamp();
+    } else {
+      // InputWatermark(op) = min { OutputWatermark(op') | op1 is upstream of op}
+      inputWatermarkMin = prevOperators.stream().map(op -> op.getOutputWatermarkTime()).min(Long::compare).get();
+    }
+
+    if (inputWatermarkTime < inputWatermarkMin) {
+      // advance the watermark time of this operator
+      inputWatermarkTime = inputWatermarkMin;
+      Watermark inputWatermark = watermark.copyWithTimestamp(inputWatermarkTime);
+      long oldestWorkTime = handleWatermark(inputWatermark, collector, coordinator);
+
+      // OutputWatermark(op) = min { InputWatermark(op), OldestWorkTime(op) }
+      long outputWatermarkMin = Math.min(inputWatermarkTime, oldestWorkTime);
+      if (outputWatermarkTime < outputWatermarkMin) {
+        // populate the watermark to downstream
+        outputWatermarkTime = outputWatermarkMin;
+        Watermark outputWatermark = watermark.copyWithTimestamp(outputWatermarkTime);
+        this.registeredOperators.forEach(op -> op.onWatermark(outputWatermark, collector, coordinator));
+      }
+    }
+  }
+
+  /**
+   * Returns the oldest time of the envelops that haven't been processed by this operator
+   * Default implementation of handling watermark, which returns the input watermark time
+   * @param inputWatermark input watermark
+   * @param collector message collector
+   * @param coordinator task coordinator
+   * @return time of oldest processing envelope
+   */
+  protected long handleWatermark(Watermark inputWatermark,
+      MessageCollector collector,
+      TaskCoordinator coordinator) {
+    return inputWatermark.getTimestamp();
+  }
+
+  /* package private */
+  long getInputWatermarkTime() {
+    return this.inputWatermarkTime;
+  }
+
+  /* package private */
+  long getOutputWatermarkTime() {
+    return this.outputWatermarkTime;
+  }
+
   public void close() {
     if (closed) {
       throw new IllegalStateException(

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/main/java/org/apache/samza/operators/impl/OutputOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/OutputOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/OutputOperatorImpl.java
index fe59b74..f212b3e 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/impl/OutputOperatorImpl.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/OutputOperatorImpl.java
@@ -18,7 +18,10 @@
  */
 package org.apache.samza.operators.impl;
 
+import java.util.Collection;
+import java.util.Collections;
 import org.apache.samza.config.Config;
+import org.apache.samza.control.Watermark;
 import org.apache.samza.operators.spec.OperatorSpec;
 import org.apache.samza.operators.spec.OutputOperatorSpec;
 import org.apache.samza.operators.spec.OutputStreamImpl;
@@ -28,9 +31,6 @@ 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.Collections;
-
 
 /**
  * An operator that sends incoming messages to an output {@link SystemStream}.
@@ -69,4 +69,14 @@ class OutputOperatorImpl<M> extends OperatorImpl<M, Void> {
   protected OperatorSpec<M, Void> getOperatorSpec() {
     return outputOpSpec;
   }
+
+  @Override
+  protected long handleWatermark(Watermark inputWatermark,
+      MessageCollector collector,
+      TaskCoordinator coordinator) {
+    if (outputOpSpec.getOpCode() == OperatorSpec.OpCode.PARTITION_BY) {
+      inputWatermark.propagate(outputStream.getStreamSpec().toSystemStream());
+    }
+    return inputWatermark.getTimestamp();
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/main/java/org/apache/samza/processor/StreamProcessor.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/processor/StreamProcessor.java b/samza-core/src/main/java/org/apache/samza/processor/StreamProcessor.java
index 14a14a8..653c0bb 100644
--- a/samza-core/src/main/java/org/apache/samza/processor/StreamProcessor.java
+++ b/samza-core/src/main/java/org/apache/samza/processor/StreamProcessor.java
@@ -36,7 +36,6 @@ import org.apache.samza.container.SamzaContainerListener;
 import org.apache.samza.coordinator.JobCoordinator;
 import org.apache.samza.coordinator.JobCoordinatorFactory;
 import org.apache.samza.coordinator.JobCoordinatorListener;
-import org.apache.samza.job.model.ContainerModel;
 import org.apache.samza.job.model.JobModel;
 import org.apache.samza.metrics.MetricsReporter;
 import org.apache.samza.task.AsyncStreamTaskFactory;
@@ -187,11 +186,11 @@ public class StreamProcessor {
 
   }
 
-  SamzaContainer createSamzaContainer(ContainerModel containerModel, int maxChangelogStreamPartitions) {
+  SamzaContainer createSamzaContainer(String processorId, JobModel jobModel) {
     return SamzaContainer.apply(
-        containerModel,
+        processorId,
+        jobModel,
         config,
-        maxChangelogStreamPartitions,
         Util.<String, MetricsReporter>javaMapAsScalaMap(customMetricsReporter),
         taskFactory);
   }
@@ -283,9 +282,7 @@ public class StreamProcessor {
             }
           };
 
-          container = createSamzaContainer(
-              jobModel.getContainers().get(processorId),
-              jobModel.maxChangeLogStreamPartitions);
+          container = createSamzaContainer(processorId, jobModel);
           container.setContainerListener(containerListener);
           LOGGER.info("Starting container " + container.toString());
           executorService = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
@@ -318,4 +315,9 @@ public class StreamProcessor {
       }
     };
   }
+
+  /* package private for testing */
+  SamzaContainer getContainer() {
+    return container;
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java
index b0bfc8a..995645d 100644
--- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java
+++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java
@@ -272,4 +272,9 @@ public class LocalApplicationRunner extends AbstractApplicationRunner {
           taskFactory.getClass().getCanonicalName()));
     }
   }
+
+  /* package private for testing */
+  Set<StreamProcessor> getProcessors() {
+    return processors;
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java
index 5d0e455..50c8181 100644
--- a/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java
+++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalContainerRunner.java
@@ -34,7 +34,6 @@ import org.apache.samza.container.SamzaContainer$;
 import org.apache.samza.container.SamzaContainerExceptionHandler;
 import org.apache.samza.container.SamzaContainerListener;
 import org.apache.samza.job.ApplicationStatus;
-import org.apache.samza.job.model.ContainerModel;
 import org.apache.samza.job.model.JobModel;
 import org.apache.samza.metrics.MetricsReporter;
 import org.apache.samza.task.TaskFactoryUtil;
@@ -73,13 +72,12 @@ public class LocalContainerRunner extends AbstractApplicationRunner {
 
   @Override
   public void run(StreamApplication streamApp) {
-    ContainerModel containerModel = jobModel.getContainers().get(containerId);
     Object taskFactory = TaskFactoryUtil.createTaskFactory(config, streamApp, this);
 
     container = SamzaContainer$.MODULE$.apply(
-        containerModel,
+        containerId,
+        jobModel,
         config,
-        jobModel.maxChangeLogStreamPartitions,
         Util.<String, MetricsReporter>javaMapAsScalaMap(new HashMap<>()),
         taskFactory);
     container.setContainerListener(

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/main/java/org/apache/samza/serializers/IntermediateMessageSerde.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/serializers/IntermediateMessageSerde.java b/samza-core/src/main/java/org/apache/samza/serializers/IntermediateMessageSerde.java
index 26ef92c..0b98ec6 100644
--- a/samza-core/src/main/java/org/apache/samza/serializers/IntermediateMessageSerde.java
+++ b/samza-core/src/main/java/org/apache/samza/serializers/IntermediateMessageSerde.java
@@ -22,7 +22,7 @@ package org.apache.samza.serializers;
 import java.util.Arrays;
 import org.apache.samza.SamzaException;
 import org.apache.samza.message.EndOfStreamMessage;
-import org.apache.samza.message.MessageType;
+import org.apache.samza.message.IntermediateMessageType;
 import org.apache.samza.message.WatermarkMessage;
 import org.codehaus.jackson.type.TypeReference;
 
@@ -86,16 +86,16 @@ public class IntermediateMessageSerde implements Serde<Object> {
   public Object fromBytes(byte[] bytes) {
     try {
       final Object object;
-      final MessageType type = MessageType.values()[bytes[0]];
+      final IntermediateMessageType type = IntermediateMessageType.values()[bytes[0]];
       final byte [] data = Arrays.copyOfRange(bytes, 1, bytes.length);
       switch (type) {
         case USER_MESSAGE:
           object = userMessageSerde.fromBytes(data);
           break;
-        case WATERMARK:
+        case WATERMARK_MESSAGE:
           object = watermarkSerde.fromBytes(data);
           break;
-        case END_OF_STREAM:
+        case END_OF_STREAM_MESSAGE:
           object = eosSerde.fromBytes(data);
           break;
         default:
@@ -117,15 +117,15 @@ public class IntermediateMessageSerde implements Serde<Object> {
   @Override
   public byte[] toBytes(Object object) {
     final byte [] data;
-    final MessageType type = MessageType.of(object);
+    final IntermediateMessageType type = IntermediateMessageType.of(object);
     switch (type) {
       case USER_MESSAGE:
         data = userMessageSerde.toBytes(object);
         break;
-      case WATERMARK:
+      case WATERMARK_MESSAGE:
         data = watermarkSerde.toBytes((WatermarkMessage) object);
         break;
-      case END_OF_STREAM:
+      case END_OF_STREAM_MESSAGE:
         data = eosSerde.toBytes((EndOfStreamMessage) object);
         break;
       default:

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/main/java/org/apache/samza/task/AsyncRunLoop.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/task/AsyncRunLoop.java b/samza-core/src/main/java/org/apache/samza/task/AsyncRunLoop.java
index e5c40df..b2903fb 100644
--- a/samza-core/src/main/java/org/apache/samza/task/AsyncRunLoop.java
+++ b/samza-core/src/main/java/org/apache/samza/task/AsyncRunLoop.java
@@ -320,7 +320,7 @@ public class AsyncRunLoop implements Runnable, Throttleable {
       this.task = task;
       this.callbackManager = new TaskCallbackManager(this, callbackTimer, callbackTimeoutMs, maxConcurrency, clock);
       Set<SystemStreamPartition> sspSet = getWorkingSSPSet(task);
-      this.state = new AsyncTaskState(task.taskName(), task.metrics(), sspSet);
+      this.state = new AsyncTaskState(task.taskName(), task.metrics(), sspSet, task.hasIntermediateStreams());
     }
 
     private void init() {
@@ -581,12 +581,14 @@ public class AsyncRunLoop implements Runnable, Throttleable {
     private final Set<SystemStreamPartition> processingSspSet;
     private final TaskName taskName;
     private final TaskInstanceMetrics taskMetrics;
+    private final boolean hasIntermediateStreams;
 
-    AsyncTaskState(TaskName taskName, TaskInstanceMetrics taskMetrics, Set<SystemStreamPartition> sspSet) {
+    AsyncTaskState(TaskName taskName, TaskInstanceMetrics taskMetrics, Set<SystemStreamPartition> sspSet, boolean hasIntermediateStreams) {
       this.taskName = taskName;
       this.taskMetrics = taskMetrics;
       this.pendingEnvelopeQueue = new ArrayDeque<>();
       this.processingSspSet = sspSet;
+      this.hasIntermediateStreams = hasIntermediateStreams;
     }
 
     private boolean checkEndOfStream() {
@@ -597,7 +599,9 @@ public class AsyncRunLoop implements Runnable, Throttleable {
         if (envelope.isEndOfStream()) {
           SystemStreamPartition ssp = envelope.getSystemStreamPartition();
           processingSspSet.remove(ssp);
-          pendingEnvelopeQueue.remove();
+          if (!hasIntermediateStreams) {
+            pendingEnvelopeQueue.remove();
+          }
         }
       }
       return processingSspSet.isEmpty();
@@ -651,7 +655,7 @@ public class AsyncRunLoop implements Runnable, Throttleable {
       if (isReady()) {
         if (needCommit) return WorkerOp.COMMIT;
         else if (needWindow) return WorkerOp.WINDOW;
-        else if (endOfStream) return WorkerOp.END_OF_STREAM;
+        else if (endOfStream && pendingEnvelopeQueue.isEmpty()) return WorkerOp.END_OF_STREAM;
         else if (!pendingEnvelopeQueue.isEmpty()) return WorkerOp.PROCESS;
       }
       return WorkerOp.NO_OP;

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/main/java/org/apache/samza/task/AsyncStreamTaskAdapter.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/task/AsyncStreamTaskAdapter.java b/samza-core/src/main/java/org/apache/samza/task/AsyncStreamTaskAdapter.java
index e2fea95..e57a89f 100644
--- a/samza-core/src/main/java/org/apache/samza/task/AsyncStreamTaskAdapter.java
+++ b/samza-core/src/main/java/org/apache/samza/task/AsyncStreamTaskAdapter.java
@@ -21,7 +21,11 @@ package org.apache.samza.task;
 
 import java.util.concurrent.ExecutorService;
 import org.apache.samza.config.Config;
+import org.apache.samza.control.ControlMessageListenerTask;
+import org.apache.samza.control.Watermark;
+import org.apache.samza.control.IOGraph;
 import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.SystemStream;
 
 
 /**
@@ -30,7 +34,7 @@ import org.apache.samza.system.IncomingMessageEnvelope;
  * the callbacks once it's done. If the thread pool is null, it follows the legacy
  * synchronous model to execute the tasks on the run loop thread.
  */
-public class AsyncStreamTaskAdapter implements AsyncStreamTask, InitableTask, WindowableTask, ClosableTask, EndOfStreamListenerTask {
+public class AsyncStreamTaskAdapter implements AsyncStreamTask, InitableTask, WindowableTask, ClosableTask, EndOfStreamListenerTask, ControlMessageListenerTask {
   private final StreamTask wrappedTask;
   private final ExecutorService executor;
 
@@ -96,4 +100,20 @@ public class AsyncStreamTaskAdapter implements AsyncStreamTask, InitableTask, Wi
       ((EndOfStreamListenerTask) wrappedTask).onEndOfStream(collector, coordinator);
     }
   }
+
+  @Override
+  public IOGraph getIOGraph() {
+    if (wrappedTask instanceof ControlMessageListenerTask) {
+      return ((ControlMessageListenerTask) wrappedTask).getIOGraph();
+    }
+    return null;
+  }
+
+  @Override
+  public void onWatermark(Watermark watermark, SystemStream stream, MessageCollector collector, TaskCoordinator coordinator) {
+    if (wrappedTask instanceof ControlMessageListenerTask) {
+      ((ControlMessageListenerTask) wrappedTask).onWatermark(watermark, stream, collector, coordinator);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java b/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java
index a77ef3b..16b7e40 100644
--- a/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java
+++ b/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java
@@ -21,10 +21,13 @@ package org.apache.samza.task;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.Config;
+import org.apache.samza.control.ControlMessageListenerTask;
+import org.apache.samza.control.Watermark;
 import org.apache.samza.operators.ContextManager;
 import org.apache.samza.operators.StreamGraphImpl;
 import org.apache.samza.operators.impl.InputOperatorImpl;
 import org.apache.samza.operators.impl.OperatorImplGraph;
+import org.apache.samza.control.IOGraph;
 import org.apache.samza.runtime.ApplicationRunner;
 import org.apache.samza.system.IncomingMessageEnvelope;
 import org.apache.samza.system.SystemStream;
@@ -36,7 +39,7 @@ import org.apache.samza.util.SystemClock;
  * A {@link StreamTask} implementation that brings all the operator API implementation components together and
  * feeds the input messages into the user-defined transformation chains in {@link StreamApplication}.
  */
-public final class StreamOperatorTask implements StreamTask, InitableTask, WindowableTask, ClosableTask {
+public final class StreamOperatorTask implements StreamTask, InitableTask, WindowableTask, ClosableTask, ControlMessageListenerTask {
 
   private final StreamApplication streamApplication;
   private final ApplicationRunner runner;
@@ -44,6 +47,7 @@ public final class StreamOperatorTask implements StreamTask, InitableTask, Windo
 
   private OperatorImplGraph operatorImplGraph;
   private ContextManager contextManager;
+  private IOGraph ioGraph;
 
   /**
    * Constructs an adaptor task to run the user-implemented {@link StreamApplication}.
@@ -87,6 +91,7 @@ public final class StreamOperatorTask implements StreamTask, InitableTask, Windo
 
     // create the operator impl DAG corresponding to the logical operator spec DAG
     this.operatorImplGraph = new OperatorImplGraph(streamGraph, config, context, clock);
+    this.ioGraph = streamGraph.toIOGraph();
   }
 
   /**
@@ -116,10 +121,31 @@ public final class StreamOperatorTask implements StreamTask, InitableTask, Windo
   }
 
   @Override
+  public IOGraph getIOGraph() {
+    return ioGraph;
+  }
+
+  @Override
+  public final void onWatermark(Watermark watermark,
+      SystemStream systemStream,
+      MessageCollector collector,
+      TaskCoordinator coordinator) {
+    InputOperatorImpl inputOpImpl = operatorImplGraph.getInputOperator(systemStream);
+    if (inputOpImpl != null) {
+      inputOpImpl.onWatermark(watermark, collector, coordinator);
+    }
+  }
+
+  @Override
   public void close() throws Exception {
     if (this.contextManager != null) {
       this.contextManager.close();
     }
     operatorImplGraph.close();
   }
+
+  /* package private for testing */
+  OperatorImplGraph getOperatorImplGraph() {
+    return this.operatorImplGraph;
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala
index 3bf5c95..481cbcf 100644
--- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala
+++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala
@@ -109,13 +109,14 @@ object SamzaContainer extends Logging {
   }
 
   def apply(
-    containerModel: ContainerModel,
+    containerId: String,
+    jobModel: JobModel,
     config: Config,
-    maxChangeLogStreamPartitions: Int,
     customReporters: Map[String, MetricsReporter] = Map[String, MetricsReporter](),
     taskFactory: Object) = {
-    val containerId = containerModel.getProcessorId()
+    val containerModel = jobModel.getContainers.get(containerId)
     val containerName = "samza-container-%s" format containerId
+    val maxChangeLogStreamPartitions = jobModel.maxChangeLogStreamPartitions
 
     var localityManager: LocalityManager = null
     if (new ClusterManagerConfig(config).getHostAffinityEnabled()) {
@@ -558,7 +559,9 @@ object SamzaContainer extends Logging {
           storageManager = storageManager,
           reporters = reporters,
           systemStreamPartitions = systemStreamPartitions,
-          exceptionHandler = TaskInstanceExceptionHandler(taskInstanceMetrics, config))
+          exceptionHandler = TaskInstanceExceptionHandler(taskInstanceMetrics, config),
+          jobModel = jobModel,
+          streamMetadataCache = streamMetadataCache)
 
       val taskInstance = createTaskInstance(task)
 
@@ -660,6 +663,8 @@ class SamzaContainer(
 
   def getStatus(): SamzaContainerStatus = status
 
+  def getTaskInstances() = taskInstances
+
   def setContainerListener(listener: SamzaContainerListener): Unit = {
     containerListener = listener
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala b/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala
index 84e993b..c14908c 100644
--- a/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala
+++ b/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala
@@ -20,14 +20,25 @@
 package org.apache.samza.container
 
 
+import com.google.common.collect.HashMultimap
+import com.google.common.collect.Multimap
 import org.apache.samza.SamzaException
 import org.apache.samza.checkpoint.OffsetManager
 import org.apache.samza.config.Config
+import org.apache.samza.config.StreamConfig.Config2Stream
+import org.apache.samza.control.ControlMessageListenerTask
+import org.apache.samza.control.ControlMessageUtils
+import org.apache.samza.control.EndOfStreamManager
+import org.apache.samza.control.WatermarkManager
+import org.apache.samza.job.model.JobModel
+import org.apache.samza.message.MessageType
 import org.apache.samza.metrics.MetricsReporter
 import org.apache.samza.storage.TaskStorageManager
 import org.apache.samza.system.IncomingMessageEnvelope
+import org.apache.samza.system.StreamMetadataCache
 import org.apache.samza.system.SystemAdmin
 import org.apache.samza.system.SystemConsumers
+import org.apache.samza.system.SystemStream
 import org.apache.samza.system.SystemStreamPartition
 import org.apache.samza.task.AsyncStreamTask
 import org.apache.samza.task.ClosableTask
@@ -42,9 +53,31 @@ import org.apache.samza.task.WindowableTask
 import org.apache.samza.util.Logging
 
 import scala.collection.JavaConverters._
+import scala.collection.JavaConversions._
+
+object TaskInstance {
+  /**
+   * Build a map from a stream to its consumer tasks
+   * @param jobModel job model which contains ssp-to-task assignment
+   * @return the map of input stream to tasks
+   */
+  def buildInputToTasks(jobModel: JobModel): Multimap[SystemStream, String] = {
+    val streamToTasks: Multimap[SystemStream, String] = HashMultimap.create[SystemStream, String]
+    if (jobModel != null) {
+      for (containerModel <- jobModel.getContainers.values) {
+        for (taskModel <- containerModel.getTasks.values) {
+          for (ssp <- taskModel.getSystemStreamPartitions) {
+            streamToTasks.put(ssp.getSystemStream, taskModel.getTaskName.toString)
+          }
+        }
+      }
+    }
+    return streamToTasks
+  }
+}
 
 class TaskInstance(
-  task: Any,
+  val task: Any,
   val taskName: TaskName,
   config: Config,
   val metrics: TaskInstanceMetrics,
@@ -56,12 +89,15 @@ class TaskInstance(
   storageManager: TaskStorageManager = null,
   reporters: Map[String, MetricsReporter] = Map(),
   val systemStreamPartitions: Set[SystemStreamPartition] = Set(),
-  val exceptionHandler: TaskInstanceExceptionHandler = new TaskInstanceExceptionHandler) extends Logging {
+  val exceptionHandler: TaskInstanceExceptionHandler = new TaskInstanceExceptionHandler,
+  jobModel: JobModel = null,
+  streamMetadataCache: StreamMetadataCache = null) extends Logging {
   val isInitableTask = task.isInstanceOf[InitableTask]
   val isWindowableTask = task.isInstanceOf[WindowableTask]
   val isEndOfStreamListenerTask = task.isInstanceOf[EndOfStreamListenerTask]
   val isClosableTask = task.isInstanceOf[ClosableTask]
   val isAsyncTask = task.isInstanceOf[AsyncStreamTask]
+  val isControlMessageListener = task.isInstanceOf[ControlMessageListenerTask]
 
   val context = new TaskContext {
     var userContext: Object = null;
@@ -93,9 +129,15 @@ class TaskInstance(
   // store the (ssp -> if this ssp is catched up) mapping. "catched up"
   // means the same ssp in other taskInstances have the same offset as
   // the one here.
-  var ssp2catchedupMapping: scala.collection.mutable.Map[SystemStreamPartition, Boolean] =
+  var ssp2CaughtupMapping: scala.collection.mutable.Map[SystemStreamPartition, Boolean] =
     scala.collection.mutable.Map[SystemStreamPartition, Boolean]()
-  systemStreamPartitions.foreach(ssp2catchedupMapping += _ -> false)
+  systemStreamPartitions.foreach(ssp2CaughtupMapping += _ -> false)
+
+  val inputToTasksMapping = TaskInstance.buildInputToTasks(jobModel)
+  var endOfStreamManager: EndOfStreamManager = null
+  var watermarkManager: WatermarkManager = null
+
+  val hasIntermediateStreams = config.getStreamIds.exists(config.getIsIntermediate(_))
 
   def registerMetrics {
     debug("Registering metrics for taskName: %s" format taskName)
@@ -127,6 +169,22 @@ class TaskInstance(
     } else {
       debug("Skipping task initialization for taskName: %s" format taskName)
     }
+
+    if (isControlMessageListener) {
+      endOfStreamManager = new EndOfStreamManager(taskName.getTaskName,
+                                                  task.asInstanceOf[ControlMessageListenerTask],
+                                                  inputToTasksMapping,
+                                                  systemStreamPartitions.asJava,
+                                                  streamMetadataCache,
+                                                  collector)
+
+      watermarkManager = new WatermarkManager(taskName.getTaskName,
+                                                  task.asInstanceOf[ControlMessageListenerTask],
+                                                  inputToTasksMapping,
+                                                  systemStreamPartitions.asJava,
+                                                  streamMetadataCache,
+                                                  collector)
+    }
   }
 
   def registerProducers {
@@ -154,31 +212,62 @@ class TaskInstance(
     callbackFactory: TaskCallbackFactory = null) {
     metrics.processes.inc
 
-    if (!ssp2catchedupMapping.getOrElse(envelope.getSystemStreamPartition,
+    if (!ssp2CaughtupMapping.getOrElse(envelope.getSystemStreamPartition,
       throw new SamzaException(envelope.getSystemStreamPartition + " is not registered!"))) {
       checkCaughtUp(envelope)
     }
 
-    if (ssp2catchedupMapping(envelope.getSystemStreamPartition)) {
+    if (ssp2CaughtupMapping(envelope.getSystemStreamPartition)) {
       metrics.messagesActuallyProcessed.inc
 
       trace("Processing incoming message envelope for taskName and SSP: %s, %s"
         format (taskName, envelope.getSystemStreamPartition))
 
-      if (isAsyncTask) {
-        exceptionHandler.maybeHandle {
-          val callback = callbackFactory.createCallback()
-          task.asInstanceOf[AsyncStreamTask].processAsync(envelope, collector, coordinator, callback)
-        }
-      } else {
-        exceptionHandler.maybeHandle {
-         task.asInstanceOf[StreamTask].process(envelope, collector, coordinator)
-        }
+      MessageType.of(envelope.getMessage) match {
+        case MessageType.USER_MESSAGE =>
+          if (isAsyncTask) {
+            exceptionHandler.maybeHandle {
+             val callback = callbackFactory.createCallback()
+             task.asInstanceOf[AsyncStreamTask].processAsync(envelope, collector, coordinator, callback)
+            }
+          }
+          else {
+            exceptionHandler.maybeHandle {
+             task.asInstanceOf[StreamTask].process(envelope, collector, coordinator)
+            }
+
+            trace("Updating offset map for taskName, SSP and offset: %s, %s, %s"
+              format(taskName, envelope.getSystemStreamPartition, envelope.getOffset))
+
+            offsetManager.update(taskName, envelope.getSystemStreamPartition, envelope.getOffset)
+          }
 
-        trace("Updating offset map for taskName, SSP and offset: %s, %s, %s"
-          format (taskName, envelope.getSystemStreamPartition, envelope.getOffset))
+        case MessageType.END_OF_STREAM =>
+          if (isControlMessageListener) {
+            // handle eos synchronously.
+            runSync(callbackFactory) {
+              endOfStreamManager.update(envelope, coordinator)
+            }
+          } else {
+            warn("Ignore end-of-stream message due to %s not implementing ControlMessageListener."
+              format(task.getClass.toString))
+          }
 
-        offsetManager.update(taskName, envelope.getSystemStreamPartition, envelope.getOffset)
+        case MessageType.WATERMARK =>
+          if (isControlMessageListener) {
+            // handle watermark synchronously in the run loop thread.
+            // we might consider running it asynchronously later
+            runSync(callbackFactory) {
+              val watermark = watermarkManager.update(envelope)
+              if (watermark != null) {
+                val stream = envelope.getSystemStreamPartition.getSystemStream
+                task.asInstanceOf[ControlMessageListenerTask].onWatermark(watermark, stream, collector, coordinator)
+              }
+            }
+          } else {
+            warn("Ignore watermark message due to %s not implementing ControlMessageListener."
+              format(task.getClass.toString))
+          }
       }
     }
   }
@@ -255,7 +344,7 @@ class TaskInstance(
     systemAdmins match {
       case null => {
         warn("systemAdmin is null. Set all SystemStreamPartitions to catched-up")
-        ssp2catchedupMapping(envelope.getSystemStreamPartition) = true
+        ssp2CaughtupMapping(envelope.getSystemStreamPartition) = true
       }
       case others => {
         val startingOffset = offsetManager.getStartingOffset(taskName, envelope.getSystemStreamPartition)
@@ -264,16 +353,26 @@ class TaskInstance(
         others(system).offsetComparator(envelope.getOffset, startingOffset) match {
           case null => {
             info("offsets in " + system + " is not comparable. Set all SystemStreamPartitions to catched-up")
-            ssp2catchedupMapping(envelope.getSystemStreamPartition) = true // not comparable
+            ssp2CaughtupMapping(envelope.getSystemStreamPartition) = true // not comparable
           }
           case result => {
             if (result >= 0) {
               info(envelope.getSystemStreamPartition.toString + " is catched up.")
-              ssp2catchedupMapping(envelope.getSystemStreamPartition) = true
+              ssp2CaughtupMapping(envelope.getSystemStreamPartition) = true
             }
           }
         }
       }
     }
   }
+
+  private def runSync(callbackFactory: TaskCallbackFactory)(runCodeBlock: => Unit) = {
+    val callback = callbackFactory.createCallback()
+    try {
+      runCodeBlock
+      callback.complete()
+    } catch {
+      case t: Throwable => callback.failure(t)
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala
index 385a060..6de4ce0 100644
--- a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala
+++ b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala
@@ -38,7 +38,7 @@ class ThreadJobFactory extends StreamJobFactory with Logging {
     info("Creating a ThreadJob, which is only meant for debugging.")
     val coordinator = JobModelManager(config)
     val jobModel = coordinator.jobModel
-    val containerModel = jobModel.getContainers.get("0")
+    val containerId = "0"
     val jmxServer = new JmxServer
     val streamApp = TaskFactoryUtil.createStreamApplication(config)
     val appRunner = new LocalContainerRunner(jobModel, "0")
@@ -66,9 +66,9 @@ class ThreadJobFactory extends StreamJobFactory with Logging {
     try {
       coordinator.start
       val container = SamzaContainer(
-        containerModel,
+        containerId,
+        jobModel,
         config,
-        jobModel.maxChangeLogStreamPartitions,
         Map[String, MetricsReporter](),
         taskFactory)
       container.setContainerListener(containerListener)

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/main/scala/org/apache/samza/system/StreamMetadataCache.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/system/StreamMetadataCache.scala b/samza-core/src/main/scala/org/apache/samza/system/StreamMetadataCache.scala
index a1b1e27..271279f 100644
--- a/samza-core/src/main/scala/org/apache/samza/system/StreamMetadataCache.scala
+++ b/samza-core/src/main/scala/org/apache/samza/system/StreamMetadataCache.scala
@@ -100,6 +100,17 @@ class StreamMetadataCache (
     allResults
   }
 
+  /**
+   * Returns metadata about the given streams. If the metadata isn't in the cache, it is retrieved from the systems
+   * using the given SystemAdmins.
+   *
+   * @param stream SystemStreams for which the metadata is requested
+   * @param partitionsMetadataOnly Flag to indicate that only partition count metadata should be fetched/refreshed
+   */
+  def getSystemStreamMetadata(stream: SystemStream, partitionsMetadataOnly: Boolean): SystemStreamMetadata = {
+    getStreamMetadata(Set(stream), partitionsMetadataOnly).get(stream).orNull
+  }
+
   private def getFromCache(stream: SystemStream, now: Long) = {
     cache.get(stream) match {
       case Some(CacheEntry(metadata, lastRefresh)) =>

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/test/java/org/apache/samza/control/TestControlMessageUtils.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/control/TestControlMessageUtils.java b/samza-core/src/test/java/org/apache/samza/control/TestControlMessageUtils.java
new file mode 100644
index 0000000..8351802
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/control/TestControlMessageUtils.java
@@ -0,0 +1,115 @@
+/*
+ * 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.control;
+
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.samza.Partition;
+import org.apache.samza.container.TaskName;
+import org.apache.samza.message.ControlMessage;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.StreamMetadataCache;
+import org.apache.samza.system.StreamSpec;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamMetadata;
+import org.apache.samza.task.MessageCollector;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
+public class TestControlMessageUtils {
+
+  @Test
+  public void testSendControlMessage() {
+    SystemStreamMetadata metadata = mock(SystemStreamMetadata.class);
+    Map<Partition, SystemStreamMetadata.SystemStreamPartitionMetadata> partitionMetadata = new HashMap<>();
+    partitionMetadata.put(new Partition(0), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
+    partitionMetadata.put(new Partition(1), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
+    partitionMetadata.put(new Partition(2), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
+    partitionMetadata.put(new Partition(3), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
+    when(metadata.getSystemStreamPartitionMetadata()).thenReturn(partitionMetadata);
+    StreamMetadataCache metadataCache = mock(StreamMetadataCache.class);
+    when(metadataCache.getSystemStreamMetadata(anyObject(), anyBoolean())).thenReturn(metadata);
+
+    SystemStream systemStream = new SystemStream("test-system", "test-stream");
+    Set<Integer> partitions = new HashSet<>();
+    MessageCollector collector = mock(MessageCollector.class);
+    doAnswer(invocation -> {
+        OutgoingMessageEnvelope envelope = (OutgoingMessageEnvelope) invocation.getArguments()[0];
+        partitions.add((Integer) envelope.getPartitionKey());
+        assertEquals(envelope.getSystemStream(), systemStream);
+        return null;
+      }).when(collector).send(any());
+
+    ControlMessageUtils.sendControlMessage(mock(ControlMessage.class), systemStream, metadataCache, collector);
+    assertEquals(partitions.size(), 4);
+  }
+
+  @Test
+  public void testCalculateUpstreamTaskCounts() {
+    SystemStream input1 = new SystemStream("test-system", "input-stream-1");
+    SystemStream input2 = new SystemStream("test-system", "input-stream-2");
+    SystemStream input3 = new SystemStream("test-system", "input-stream-3");
+
+    Multimap<SystemStream, String> inputToTasks = HashMultimap.create();
+    TaskName t0 = new TaskName("task 0"); //consume input1 and input2
+    TaskName t1 = new TaskName("task 1"); //consume input1 and input2 and input 3
+    TaskName t2 = new TaskName("task 2"); //consume input2 and input 3
+    inputToTasks.put(input1, t0.getTaskName());
+    inputToTasks.put(input1, t1.getTaskName());
+    inputToTasks.put(input2, t0.getTaskName());
+    inputToTasks.put(input2, t1.getTaskName());
+    inputToTasks.put(input2, t2.getTaskName());
+    inputToTasks.put(input3, t1.getTaskName());
+    inputToTasks.put(input3, t2.getTaskName());
+
+    StreamSpec inputSpec2 = new StreamSpec("input-stream-2", "input-stream-2", "test-system");
+    StreamSpec inputSpec3 = new StreamSpec("input-stream-3", "input-stream-3", "test-system");
+    StreamSpec intSpec1 = new StreamSpec("int-stream-1", "int-stream-1", "test-system");
+    StreamSpec intSpec2 = new StreamSpec("int-stream-2", "int-stream-2", "test-system");
+
+    List<IOGraph.IONode> nodes = new ArrayList<>();
+    IOGraph.IONode node = new IOGraph.IONode(intSpec1, true);
+    node.addInput(inputSpec2);
+    nodes.add(node);
+    node = new IOGraph.IONode(intSpec2, true);
+    node.addInput(inputSpec3);
+    nodes.add(node);
+    IOGraph ioGraph = new IOGraph(nodes);
+
+    Map<SystemStream, Integer> counts = ControlMessageUtils.calculateUpstreamTaskCounts(inputToTasks, ioGraph);
+    assertEquals(counts.get(intSpec1.toSystemStream()).intValue(), 3);
+    assertEquals(counts.get(intSpec2.toSystemStream()).intValue(), 2);
+  }
+
+}
\ No newline at end of file


[06/16] samza git commit: SAMZA-1415: Add clearStream API in SystemAdmin and remove deprecated APIs

Posted by xi...@apache.org.
SAMZA-1415: Add clearStream API in SystemAdmin and remove deprecated APIs

The patch does the following:

1) add clearStream() APi in SystemAdmin. Currently it's only supported in Kafka with broker configuring delete.topic.enable=true.

2) remove the deprecated APIs including createChangeLogStream(), validateChangelogStream() and createCoordinatorStream().

Author: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>

Reviewers: Jake Maes <ja...@gmail.com>

Closes #292 from xinyuiscool/SAMZA-1415


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

Branch: refs/heads/master
Commit: 79200c73509727abba2d6eb68cfc45ac1d842d35
Parents: 23bfaa8
Author: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>
Authored: Thu Sep 7 16:49:20 2017 -0700
Committer: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>
Committed: Thu Sep 7 16:49:20 2017 -0700

----------------------------------------------------------------------
 .../org/apache/samza/system/StreamSpec.java     |  22 +++
 .../org/apache/samza/system/SystemAdmin.java    |  42 ++----
 ...inglePartitionWithoutOffsetsSystemAdmin.java |  16 --
 .../samza/coordinator/JobModelManager.scala     |   8 +-
 .../scala/org/apache/samza/job/JobRunner.scala  |   9 +-
 .../samza/storage/TaskStorageManager.scala      |  12 +-
 .../MockCoordinatorStreamSystemFactory.java     |  13 +-
 .../samza/execution/TestExecutionPlanner.java   |  15 --
 .../samza/checkpoint/TestOffsetManager.scala    |  12 --
 .../samza/container/TestTaskInstance.scala      |   3 -
 .../samza/coordinator/TestJobCoordinator.scala  |  12 --
 .../samza/storage/TestTaskStorageManager.scala  |   6 +-
 .../elasticsearch/ElasticsearchSystemAdmin.java |  15 --
 .../samza/system/hdfs/HdfsSystemAdmin.java      |  15 --
 .../samza/system/kafka/KafkaSystemAdmin.scala   |  93 +++++++-----
 .../system/kafka/TestKafkaSystemAdminJava.java  | 151 ++++++++++---------
 .../system/kafka/TestKafkaSystemAdmin.scala     |   5 +-
 .../samza/system/mock/MockSystemAdmin.java      |  15 --
 .../samza/test/util/SimpleSystemAdmin.java      |  15 --
 .../apache/samza/job/yarn/MockSystemAdmin.scala |  12 --
 20 files changed, 192 insertions(+), 299 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/79200c73/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java b/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
index 49531dd..384fecc 100644
--- a/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
+++ b/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
@@ -37,6 +37,12 @@ public class StreamSpec {
 
   private static final int DEFAULT_PARTITION_COUNT = 1;
 
+  // Internal changelog stream id. It is used for creating changelog StreamSpec.
+  private static final String CHANGELOG_STREAM_ID = "samza-internal-changelog-stream-id";
+
+  // Internal coordinator stream id. It is used for creating coordinator StreamSpec.
+  private static final String COORDINATOR_STREAM_ID = "samza-internal-coordinator-stream-id";
+
   /**
    * Unique identifier for the stream in a Samza application.
    * This identifier is used as a key for stream properties in the
@@ -200,6 +206,14 @@ public class StreamSpec {
     return new SystemStream(systemName, physicalName);
   }
 
+  public boolean isChangeLogStream() {
+    return id.equals(CHANGELOG_STREAM_ID);
+  }
+
+  public boolean isCoordinatorStream() {
+    return id.equals(COORDINATOR_STREAM_ID);
+  }
+
   private void validateLogicalIdentifier(String identifierName, String identifierValue) {
     if (identifierValue == null || !identifierValue.matches("[A-Za-z0-9_-]+")) {
       throw new IllegalArgumentException(String.format("Identifier '%s' is '%s'. It must match the expression [A-Za-z0-9_-]+", identifierName, identifierValue));
@@ -220,4 +234,12 @@ public class StreamSpec {
   public int hashCode() {
     return id.hashCode();
   }
+
+  public static StreamSpec createChangeLogStreamSpec(String physicalName, String systemName, int partitionCount) {
+    return new StreamSpec(CHANGELOG_STREAM_ID, physicalName, systemName, partitionCount);
+  }
+
+  public static StreamSpec createCoordinatorStreamSpec(String physicalName, String systemName) {
+    return new StreamSpec(COORDINATOR_STREAM_ID, physicalName, systemName, 1);
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/79200c73/samza-api/src/main/java/org/apache/samza/system/SystemAdmin.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/SystemAdmin.java b/samza-api/src/main/java/org/apache/samza/system/SystemAdmin.java
index b180712..e765540 100644
--- a/samza-api/src/main/java/org/apache/samza/system/SystemAdmin.java
+++ b/samza-api/src/main/java/org/apache/samza/system/SystemAdmin.java
@@ -50,38 +50,6 @@ public interface SystemAdmin {
   Map<String, SystemStreamMetadata> getSystemStreamMetadata(Set<String> streamNames);
 
   /**
-   * An API to create a change log stream
-   *
-   * @param streamName
-   *          The name of the stream to be created in the underlying stream
-   * @param numOfPartitions
-   *          The number of partitions in the changelog stream
-   * @deprecated since 0.12.1, use {@link #createStream(StreamSpec)}
-   */
-  void createChangelogStream(String streamName, int numOfPartitions);
-
-  /**
-   * Validates change log stream
-   *
-   * @param streamName
-   *          The name of the stream to be created in the underlying stream
-   * @param numOfPartitions
-   *          The number of partitions in the changelog stream
-   * @deprecated since 0.12.1, use {@link #validateStream(StreamSpec)}
-   */
-  void validateChangelogStream(String streamName, int numOfPartitions);
-
-  /**
-   * Create a stream for the job coordinator. If the stream already exists, this
-   * call should simply return.
-   *
-   * @param streamName
-   *          The name of the coordinator stream to create.
-   * @deprecated since 0.12.1, use {@link #createStream(StreamSpec)}
-   */
-  void createCoordinatorStream(String streamName);
-
-  /**
    * Compare the two offsets. -1, 0, +1 means offset1 &lt; offset2,
    * offset1 == offset2 and offset1 &gt; offset2 respectively. Return
    * null if those two offsets are not comparable
@@ -114,4 +82,14 @@ public interface SystemAdmin {
   default void validateStream(StreamSpec streamSpec) throws StreamValidationException {
     throw new UnsupportedOperationException();
   }
+
+  /**
+   * Clear the stream described by the spec.
+   * @param streamSpec  The spec for the physical stream on the system.
+   * @return {@code true} if the stream was successfully cleared.
+   *         {@code false} if clearing stream failed.
+   */
+  default boolean clearStream(StreamSpec streamSpec) {
+    throw new UnsupportedOperationException();
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/79200c73/samza-api/src/main/java/org/apache/samza/util/SinglePartitionWithoutOffsetsSystemAdmin.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/util/SinglePartitionWithoutOffsetsSystemAdmin.java b/samza-api/src/main/java/org/apache/samza/util/SinglePartitionWithoutOffsetsSystemAdmin.java
index 2157e69..49f7da0 100644
--- a/samza-api/src/main/java/org/apache/samza/util/SinglePartitionWithoutOffsetsSystemAdmin.java
+++ b/samza-api/src/main/java/org/apache/samza/util/SinglePartitionWithoutOffsetsSystemAdmin.java
@@ -24,7 +24,6 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.samza.Partition;
-import org.apache.samza.SamzaException;
 import org.apache.samza.system.SystemAdmin;
 import org.apache.samza.system.SystemStreamMetadata;
 import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata;
@@ -56,16 +55,6 @@ public class SinglePartitionWithoutOffsetsSystemAdmin implements SystemAdmin {
   }
 
   @Override
-  public void createChangelogStream(String streamName, int numOfPartitions) {
-    throw new SamzaException("Method not implemented");
-  }
-
-  @Override
-  public void validateChangelogStream(String streamName, int numOfPartitions) {
-    throw new SamzaException("Method not implemented");
-  }
-
-  @Override
   public Map<SystemStreamPartition, String> getOffsetsAfter(Map<SystemStreamPartition, String> offsets) {
     Map<SystemStreamPartition, String> offsetsAfter = new HashMap<SystemStreamPartition, String>();
 
@@ -77,11 +66,6 @@ public class SinglePartitionWithoutOffsetsSystemAdmin implements SystemAdmin {
   }
 
   @Override
-  public void createCoordinatorStream(String streamName) {
-    throw new UnsupportedOperationException("Single partition admin can't create coordinator streams.");
-  }
-
-  @Override
   public Integer offsetComparator(String offset1, String offset2) {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/79200c73/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala b/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala
index 6319173..42bedec 100644
--- a/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala
+++ b/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala
@@ -304,7 +304,13 @@ object JobModelManager extends Logging {
         .getOrElse(throw new SamzaException("A stream uses system %s, which is missing from the configuration." format systemStream.getSystem))
         ).getAdmin(systemStream.getSystem, config)
 
-      systemAdmin.createChangelogStream(systemStream.getStream, changeLogPartitions)
+      val changelogSpec = StreamSpec.createChangeLogStreamSpec(systemStream.getStream, systemStream.getSystem, changeLogPartitions)
+      if (systemAdmin.createStream(changelogSpec)) {
+        info("Created changelog stream %s." format systemStream.getStream)
+      } else {
+        info("Changelog stream %s already exists." format systemStream.getStream)
+      }
+      systemAdmin.validateStream(changelogSpec)
     }
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/79200c73/samza-core/src/main/scala/org/apache/samza/job/JobRunner.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/job/JobRunner.scala b/samza-core/src/main/scala/org/apache/samza/job/JobRunner.scala
index f34db99..0e973e9 100644
--- a/samza-core/src/main/scala/org/apache/samza/job/JobRunner.scala
+++ b/samza-core/src/main/scala/org/apache/samza/job/JobRunner.scala
@@ -29,6 +29,7 @@ import org.apache.samza.job.ApplicationStatus.{Running, SuccessfulFinish}
 import org.apache.samza.metrics.MetricsRegistryMap
 import org.apache.samza.runtime.ApplicationRunnerMain.ApplicationRunnerCommandLine
 import org.apache.samza.runtime.ApplicationRunnerOperation
+import org.apache.samza.system.StreamSpec
 import org.apache.samza.util.{Logging, Util}
 
 import scala.collection.JavaConverters._
@@ -85,7 +86,13 @@ class JobRunner(config: Config) extends Logging {
     info("Creating coordinator stream")
     val (coordinatorSystemStream, systemFactory) = Util.getCoordinatorSystemStreamAndFactory(config)
     val systemAdmin = systemFactory.getAdmin(coordinatorSystemStream.getSystem, config)
-    systemAdmin.createCoordinatorStream(coordinatorSystemStream.getStream)
+    val streamName = coordinatorSystemStream.getStream
+    val coordinatorSpec = StreamSpec.createCoordinatorStreamSpec(streamName, coordinatorSystemStream.getSystem)
+    if (systemAdmin.createStream(coordinatorSpec)) {
+      info("Created coordinator stream %s." format streamName)
+    } else {
+      info("Coordinator stream %s already exists." format streamName)
+    }
 
     if (resetJobConfig) {
       info("Storing config in coordinator stream.")

http://git-wip-us.apache.org/repos/asf/samza/blob/79200c73/samza-core/src/main/scala/org/apache/samza/storage/TaskStorageManager.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/storage/TaskStorageManager.scala b/samza-core/src/main/scala/org/apache/samza/storage/TaskStorageManager.scala
index 977ac5b..0879e9a 100644
--- a/samza-core/src/main/scala/org/apache/samza/storage/TaskStorageManager.scala
+++ b/samza-core/src/main/scala/org/apache/samza/storage/TaskStorageManager.scala
@@ -25,14 +25,7 @@ import java.util
 import org.apache.samza.config.StorageConfig
 import org.apache.samza.{Partition, SamzaException}
 import org.apache.samza.container.TaskName
-import org.apache.samza.system.StreamMetadataCache
-import org.apache.samza.system.SystemAdmin
-import org.apache.samza.system.SystemConsumer
-import org.apache.samza.system.SystemStream
-import org.apache.samza.system.SystemStreamPartition
-import org.apache.samza.system.SystemStreamPartitionIterator
-import org.apache.samza.system.ExtendedSystemAdmin
-import org.apache.samza.system.SystemStreamMetadata
+import org.apache.samza.system._
 import org.apache.samza.util.Logging
 import org.apache.samza.util.Util
 import org.apache.samza.util.Clock
@@ -218,7 +211,8 @@ class TaskStorageManager(
       val systemAdmin = systemAdmins
         .getOrElse(systemStream.getSystem,
                    throw new SamzaException("Unable to get systemAdmin for store " + storeName + " and systemStream" + systemStream))
-      systemAdmin.validateChangelogStream(systemStream.getStream, changeLogStreamPartitions)
+      val changelogSpec = StreamSpec.createChangeLogStreamSpec(systemStream.getStream, systemStream.getSystem, changeLogStreamPartitions)
+      systemAdmin.validateStream(changelogSpec)
     }
 
     val changeLogMetadata = streamMetadataCache.getStreamMetadata(changeLogSystemStreams.values.toSet)

http://git-wip-us.apache.org/repos/asf/samza/blob/79200c73/samza-core/src/test/java/org/apache/samza/coordinator/stream/MockCoordinatorStreamSystemFactory.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/coordinator/stream/MockCoordinatorStreamSystemFactory.java b/samza-core/src/test/java/org/apache/samza/coordinator/stream/MockCoordinatorStreamSystemFactory.java
index 662c737..6413413 100644
--- a/samza-core/src/test/java/org/apache/samza/coordinator/stream/MockCoordinatorStreamSystemFactory.java
+++ b/samza-core/src/test/java/org/apache/samza/coordinator/stream/MockCoordinatorStreamSystemFactory.java
@@ -25,14 +25,7 @@ import org.apache.samza.config.ConfigException;
 import org.apache.samza.coordinator.stream.messages.CoordinatorStreamMessage;
 import org.apache.samza.metrics.MetricsRegistry;
 import org.apache.samza.serializers.JsonSerde;
-import org.apache.samza.system.SystemAdmin;
-import org.apache.samza.system.SystemConsumer;
-import org.apache.samza.system.SystemFactory;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.system.SystemStreamPartition;
-import org.apache.samza.system.SystemProducer;
-import org.apache.samza.system.OutgoingMessageEnvelope;
-import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.*;
 import org.apache.samza.util.SinglePartitionWithoutOffsetsSystemAdmin;
 import org.apache.samza.util.Util;
 
@@ -208,8 +201,10 @@ public class MockCoordinatorStreamSystemFactory implements SystemFactory {
   }
 
   public static final class MockSystemAdmin extends SinglePartitionWithoutOffsetsSystemAdmin implements SystemAdmin {
-    public void createCoordinatorStream(String streamName) {
+    @Override
+    public boolean createStream(StreamSpec streamSpec) {
       // Do nothing.
+      return true;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/79200c73/samza-core/src/test/java/org/apache/samza/execution/TestExecutionPlanner.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestExecutionPlanner.java b/samza-core/src/test/java/org/apache/samza/execution/TestExecutionPlanner.java
index 2c8f682..c4fd8f7 100644
--- a/samza-core/src/test/java/org/apache/samza/execution/TestExecutionPlanner.java
+++ b/samza-core/src/test/java/org/apache/samza/execution/TestExecutionPlanner.java
@@ -93,21 +93,6 @@ public class TestExecutionPlanner {
       }
 
       @Override
-      public void createChangelogStream(String streamName, int numOfPartitions) {
-
-      }
-
-      @Override
-      public void validateChangelogStream(String streamName, int numOfPartitions) {
-
-      }
-
-      @Override
-      public void createCoordinatorStream(String streamName) {
-
-      }
-
-      @Override
       public Integer offsetComparator(String offset1, String offset2) {
         return null;
       }

http://git-wip-us.apache.org/repos/asf/samza/blob/79200c73/samza-core/src/test/scala/org/apache/samza/checkpoint/TestOffsetManager.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/checkpoint/TestOffsetManager.scala b/samza-core/src/test/scala/org/apache/samza/checkpoint/TestOffsetManager.scala
index abfc63f..48504a9 100644
--- a/samza-core/src/test/scala/org/apache/samza/checkpoint/TestOffsetManager.scala
+++ b/samza-core/src/test/scala/org/apache/samza/checkpoint/TestOffsetManager.scala
@@ -340,18 +340,6 @@ class TestOffsetManager {
       def getSystemStreamMetadata(streamNames: java.util.Set[String]) =
         Map[String, SystemStreamMetadata]().asJava
 
-      override def createChangelogStream(topicName: String, numOfChangeLogPartitions: Int) {
-        new UnsupportedOperationException("Method not implemented.")
-      }
-
-      override def validateChangelogStream(topicName: String, numOfChangeLogPartitions: Int) {
-        new UnsupportedOperationException("Method not implemented.")
-      }
-
-      override def createCoordinatorStream(streamName: String) {
-        new UnsupportedOperationException("Method not implemented.")
-      }
-
       override def offsetComparator(offset1: String, offset2: String) = null
     }
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/79200c73/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala b/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
index 9025077..dcb06d3 100644
--- a/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
+++ b/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
@@ -398,9 +398,6 @@ class TestTaskInstance {
 class MockSystemAdmin extends SystemAdmin {
   override def getOffsetsAfter(offsets: java.util.Map[SystemStreamPartition, String]) = { offsets }
   override def getSystemStreamMetadata(streamNames: java.util.Set[String]) = null
-  override def createCoordinatorStream(stream: String) = {}
-  override def createChangelogStream(topicName: String, numKafkaChangelogPartitions: Int) = {}
-  override def validateChangelogStream(topicName: String, numOfPartitions: Int) = {}
 
   override def offsetComparator(offset1: String, offset2: String) = {
     offset1.toLong compare offset2.toLong

http://git-wip-us.apache.org/repos/asf/samza/blob/79200c73/samza-core/src/test/scala/org/apache/samza/coordinator/TestJobCoordinator.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/coordinator/TestJobCoordinator.scala b/samza-core/src/test/scala/org/apache/samza/coordinator/TestJobCoordinator.scala
index 0b6dd8b..e6b148b 100644
--- a/samza-core/src/test/scala/org/apache/samza/coordinator/TestJobCoordinator.scala
+++ b/samza-core/src/test/scala/org/apache/samza/coordinator/TestJobCoordinator.scala
@@ -318,18 +318,6 @@ class MockSystemAdmin extends ExtendedSystemAdmin {
     Map(streamNames.asScala.toList.head -> new SystemStreamMetadata("foo", partitionMetadata.asJava)).asJava
   }
 
-  override def createChangelogStream(topicName: String, numOfChangeLogPartitions: Int) {
-    new UnsupportedOperationException("Method not implemented.")
-  }
-
-  override def validateChangelogStream(topicName: String, numOfChangeLogPartitions: Int) {
-    new UnsupportedOperationException("Method not implemented.")
-  }
-
-  override def createCoordinatorStream(streamName: String) {
-    new UnsupportedOperationException("Method not implemented.")
-  }
-
   override def offsetComparator(offset1: String, offset2: String) = null
 
   override def getSystemStreamPartitionCounts(streamNames: util.Set[String],

http://git-wip-us.apache.org/repos/asf/samza/blob/79200c73/samza-core/src/test/scala/org/apache/samza/storage/TestTaskStorageManager.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/storage/TestTaskStorageManager.scala b/samza-core/src/test/scala/org/apache/samza/storage/TestTaskStorageManager.scala
index 2495baf..ea4d37b 100644
--- a/samza-core/src/test/scala/org/apache/samza/storage/TestTaskStorageManager.scala
+++ b/samza-core/src/test/scala/org/apache/samza/storage/TestTaskStorageManager.scala
@@ -95,7 +95,8 @@ class TestTaskStorageManager extends MockitoSugar {
     val mockStreamMetadataCache = mock[StreamMetadataCache]
     val mockSystemConsumer = mock[SystemConsumer]
     val mockSystemAdmin = mock[SystemAdmin]
-    doNothing().when(mockSystemAdmin).validateChangelogStream("testStream", 1)
+    val changelogSpec = StreamSpec.createChangeLogStreamSpec("testStream", "kafka", 1)
+    doNothing().when(mockSystemAdmin).validateStream(changelogSpec)
     var registerOffset = "0"
     when(mockSystemConsumer.register(any(), any())).thenAnswer(new Answer[Unit] {
       override def answer(invocation: InvocationOnMock): Unit = {
@@ -204,7 +205,8 @@ class TestTaskStorageManager extends MockitoSugar {
     // Mock for StreamMetadataCache, SystemConsumer, SystemAdmin
     val mockStreamMetadataCache = mock[StreamMetadataCache]
     val mockSystemAdmin = mock[SystemAdmin]
-    doNothing().when(mockSystemAdmin).validateChangelogStream("testStream", 1)
+    val changelogSpec = StreamSpec.createChangeLogStreamSpec("testStream", "kafka", 1)
+    doNothing().when(mockSystemAdmin).validateStream(changelogSpec)
 
     val mockSystemConsumer = mock[SystemConsumer]
     when(mockSystemConsumer.register(any(), any())).thenAnswer(new Answer[Unit] {

http://git-wip-us.apache.org/repos/asf/samza/blob/79200c73/samza-elasticsearch/src/main/java/org/apache/samza/system/elasticsearch/ElasticsearchSystemAdmin.java
----------------------------------------------------------------------
diff --git a/samza-elasticsearch/src/main/java/org/apache/samza/system/elasticsearch/ElasticsearchSystemAdmin.java b/samza-elasticsearch/src/main/java/org/apache/samza/system/elasticsearch/ElasticsearchSystemAdmin.java
index 446534a..3cadce0 100644
--- a/samza-elasticsearch/src/main/java/org/apache/samza/system/elasticsearch/ElasticsearchSystemAdmin.java
+++ b/samza-elasticsearch/src/main/java/org/apache/samza/system/elasticsearch/ElasticsearchSystemAdmin.java
@@ -54,21 +54,6 @@ public class ElasticsearchSystemAdmin implements SystemAdmin {
   }
 
   @Override
-  public void createChangelogStream(String stream, int foo) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void createCoordinatorStream(String streamName) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void validateChangelogStream(String streamName, int numOfPartitions) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
   public Integer offsetComparator(String offset1, String offset2) {
 	  throw new UnsupportedOperationException();
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/79200c73/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemAdmin.java
----------------------------------------------------------------------
diff --git a/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemAdmin.java b/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemAdmin.java
index f5b05fb..9251db0 100644
--- a/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemAdmin.java
+++ b/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemAdmin.java
@@ -201,21 +201,6 @@ public class HdfsSystemAdmin implements SystemAdmin {
     return systemStreamMetadataMap;
   }
 
-  @Override
-  public void createChangelogStream(String streamName, int numOfPartitions) {
-    throw new UnsupportedOperationException("HDFS doesn't support change log stream.");
-  }
-
-  @Override
-  public void validateChangelogStream(String streamName, int numOfPartitions) {
-    throw new UnsupportedOperationException("HDFS doesn't support change log stream.");
-  }
-
-  @Override
-  public void createCoordinatorStream(String streamName) {
-    throw new UnsupportedOperationException("HDFS doesn't support coordinator stream.");
-  }
-
   /**
    * Compare two multi-file style offset. A multi-file style offset consist of both
    * the file index as well as the offset within that file. And the format of it is:

http://git-wip-us.apache.org/repos/asf/samza/blob/79200c73/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala
index af77d5b..6e582e9 100644
--- a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala
@@ -37,9 +37,8 @@ import scala.collection.JavaConverters._
 
 
 object KafkaSystemAdmin extends Logging {
-  // Use a dummy string for the stream id. The physical name and partition count are all that matter for changelog creation, so the dummy string should not be used.
-  // We cannot use the topic name, as it may include special chars which are not allowed in stream IDs. See SAMZA-1317
-  val CHANGELOG_STREAMID = "unused-temp-changelog-stream-id"
+
+  val CLEAR_STREAM_RETRIES = 3
 
   /**
    * A helper method that takes oldest, newest, and upcoming offsets for each
@@ -328,23 +327,11 @@ class KafkaSystemAdmin(
      offset
   }
 
-  override def createCoordinatorStream(streamName: String) {
-    info("Attempting to create coordinator stream %s." format streamName)
-
-    val streamSpec = new KafkaStreamSpec(streamName, streamName, systemName, 1, coordinatorStreamReplicationFactor, coordinatorStreamProperties)
-
-    if (createStream(streamSpec)) {
-      info("Created coordinator stream %s." format streamName)
-    } else {
-      info("Coordinator stream %s already exists." format streamName)
-    }
-  }
-
   /**
    * Helper method to use topic metadata cache when fetching metadata, so we
    * don't hammer Kafka more than we need to.
    */
-  protected def getTopicMetadata(topics: Set[String]) = {
+  def getTopicMetadata(topics: Set[String]) = {
     new ClientUtilTopicMetadataStore(brokerListString, clientId, timeout)
       .getTopicInfo(topics)
   }
@@ -415,7 +402,7 @@ class KafkaSystemAdmin(
    * @inheritdoc
    */
   override def createStream(spec: StreamSpec): Boolean = {
-    val kSpec = KafkaStreamSpec.fromSpec(spec);
+    val kSpec = toKafkaSpec(spec)
     var streamCreated = false
 
     new ExponentialSleepStrategy(initialDelayMs = 500).run(
@@ -451,6 +438,23 @@ class KafkaSystemAdmin(
   }
 
   /**
+   * Converts a StreamSpec into a KafakStreamSpec. Special handling for coordinator and changelog stream.
+   * @param spec a StreamSpec object
+   * @return KafkaStreamSpec object
+   */
+  def toKafkaSpec(spec: StreamSpec): KafkaStreamSpec = {
+    if (spec.isChangeLogStream) {
+      val topicName = spec.getPhysicalName
+      val topicMeta = topicMetaInformation.getOrElse(topicName, throw new StreamValidationException("Unable to find topic information for topic " + topicName))
+      new KafkaStreamSpec(spec.getId, topicName, systemName, spec.getPartitionCount, topicMeta.replicationFactor, topicMeta.kafkaProps)
+    } else if (spec.isCoordinatorStream){
+      new KafkaStreamSpec(spec.getId, spec.getPhysicalName, systemName, 1, coordinatorStreamReplicationFactor, coordinatorStreamProperties)
+    } else {
+      KafkaStreamSpec.fromSpec(spec)
+    }
+  }
+
+  /**
     * @inheritdoc
     *
     * Validates a stream in Kafka. Should not be called before createStream(),
@@ -491,32 +495,41 @@ class KafkaSystemAdmin(
   }
 
   /**
-    * Exception to be thrown when the change log stream creation or validation has failed
-    */
-  class KafkaChangelogException(s: String, t: Throwable) extends SamzaException(s, t) {
-    def this(s: String) = this(s, null)
-  }
-  
-  override def createChangelogStream(topicName: String, numKafkaChangelogPartitions: Int) = {
-    val topicMeta = topicMetaInformation.getOrElse(topicName, throw new KafkaChangelogException("Unable to find topic information for topic " + topicName))
-    val spec = new KafkaStreamSpec(CHANGELOG_STREAMID, topicName, systemName, numKafkaChangelogPartitions, topicMeta.replicationFactor, topicMeta.kafkaProps)
+   * @inheritdoc
+   *
+   * Delete a stream in Kafka. Deleting topics works only when the broker is configured with "delete.topic.enable=true".
+   * Otherwise it's a no-op.
+   */
+  override def clearStream(spec: StreamSpec): Boolean = {
+    val kSpec = KafkaStreamSpec.fromSpec(spec)
+    var retries = CLEAR_STREAM_RETRIES
+    new ExponentialSleepStrategy().run(
+      loop => {
+        val zkClient = connectZk()
+        try {
+          AdminUtils.deleteTopic(
+            zkClient,
+            kSpec.getPhysicalName)
+        } finally {
+          zkClient.close
+        }
 
-    if (createStream(spec)) {
-      info("Created changelog stream %s." format topicName)
-    } else {
-      info("Changelog stream %s already exists." format topicName)
-    }
+        loop.done
+      },
 
-    validateStream(spec)
-  }
+      (exception, loop) => {
+        if (retries > 0) {
+          warn("Exception while trying to delete topic %s: %s. Retrying." format (spec.getPhysicalName, exception))
+          retries -= 1
+        } else {
+          warn("Fail to delete topic %s: %s" format (spec.getPhysicalName, exception))
+          loop.done
+          throw exception
+        }
+      })
 
-  /**
-    * Validates a stream in Kafka. Should not be called before createStream(),
-    * since ClientUtils.fetchTopicMetadata(), used by different Kafka clients, is not read-only and
-    * will auto-create a new topic.
-    */
-  override def validateChangelogStream(topicName: String, numKafkaChangelogPartitions: Int) = {
-    validateStream(new KafkaStreamSpec(CHANGELOG_STREAMID, topicName, systemName, numKafkaChangelogPartitions))
+    val topicMetadata = getTopicMetadata(Set(kSpec.getPhysicalName)).get(kSpec.getPhysicalName).get
+    topicMetadata.partitionsMetadata.isEmpty
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/samza/blob/79200c73/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java b/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java
index ce59b40..51af518 100644
--- a/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java
+++ b/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java
@@ -19,15 +19,16 @@
 
 package org.apache.samza.system.kafka;
 
+import java.util.*;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.Properties;
+
+import kafka.api.TopicMetadata;
 import org.apache.samza.system.StreamSpec;
 import org.apache.samza.system.StreamValidationException;
 import org.apache.samza.system.SystemAdmin;
 import org.apache.samza.util.Util;
 import org.junit.Test;
-import org.mockito.ArgumentCaptor;
 import org.mockito.Mockito;
 
 import static org.junit.Assert.*;
@@ -39,53 +40,48 @@ public class TestKafkaSystemAdminJava extends TestKafkaSystemAdmin {
 
 
   @Test
-  public void testCreateCoordinatorStreamDelegatesToCreateStream() {
+  public void testCreateCoordinatorStream() {
     KafkaSystemAdmin systemAdmin = createSystemAdmin();//coordProps, 3, new scala.collection.immutable.HashMap<>(), 1000);
     SystemAdmin admin = Mockito.spy(systemAdmin);
-    StreamSpec spec = new StreamSpec("testId", "testCoordinatorStream", "testSystem");
+    StreamSpec spec = StreamSpec.createCoordinatorStreamSpec("testCoordinatorStream", "testSystem");
 
-    admin.createCoordinatorStream(spec.getPhysicalName());
+    admin.createStream(spec);
     admin.validateStream(spec);
 
     Mockito.verify(admin).createStream(Mockito.any());
   }
 
   @Test
-  public void testCreateChangelogStreamDelegatesToCreateStream() {
-    final String STREAM = "testChangeLogStream";
-    final int PARTITIONS = 12;
-    final int REP_FACTOR = 3;
+  public void testCreateCoordinatorStreamWithSpecialCharsInTopicName() {
+    final String STREAM = "test.coordinator_test.Stream";
 
     Properties coordProps = new Properties();
-    Properties changeLogProps = new Properties();
-    changeLogProps.setProperty("cleanup.policy", "compact");
-    changeLogProps.setProperty("segment.bytes", "139");
     Map<String, ChangelogInfo> changeLogMap = new HashMap<>();
-    changeLogMap.put(STREAM, new ChangelogInfo(REP_FACTOR, changeLogProps));
 
-    SystemAdmin admin = Mockito.spy(createSystemAdmin(coordProps, 3, Util.javaMapAsScalaMap(changeLogMap)));
-    StreamSpec spec = new StreamSpec(KafkaSystemAdmin.CHANGELOG_STREAMID(), STREAM, SYSTEM(), PARTITIONS);
-    admin.createChangelogStream(STREAM, PARTITIONS);
+    KafkaSystemAdmin admin = Mockito.spy(createSystemAdmin(coordProps, 1, Util.javaMapAsScalaMap(changeLogMap)));
+    StreamSpec spec = StreamSpec.createCoordinatorStreamSpec(STREAM, SYSTEM());
+
+    Mockito.doAnswer(invocationOnMock -> {
+      StreamSpec internalSpec = (StreamSpec) invocationOnMock.callRealMethod();
+      assertTrue(internalSpec instanceof KafkaStreamSpec);  // KafkaStreamSpec is used to carry replication factor
+      assertTrue(internalSpec.isCoordinatorStream());
+      assertEquals(SYSTEM(), internalSpec.getSystemName());
+      assertEquals(STREAM, internalSpec.getPhysicalName());
+      assertEquals(1, internalSpec.getPartitionCount());
+
+      return internalSpec;
+    }).when(admin).toKafkaSpec(Mockito.any());
+
+    admin.createStream(spec);
     admin.validateStream(spec);
 
-    ArgumentCaptor<StreamSpec> specCaptor = ArgumentCaptor.forClass(StreamSpec.class);
-    Mockito.verify(admin).createStream(specCaptor.capture());
-
-    StreamSpec internalSpec = specCaptor.getValue();
-    assertTrue(internalSpec instanceof KafkaStreamSpec);  // KafkaStreamSpec is used to carry replication factor
-    assertEquals(KafkaSystemAdmin.CHANGELOG_STREAMID(), internalSpec.getId());
-    assertEquals(SYSTEM(), internalSpec.getSystemName());
-    assertEquals(STREAM, internalSpec.getPhysicalName());
-    assertEquals(REP_FACTOR, ((KafkaStreamSpec) internalSpec).getReplicationFactor());
-    assertEquals(PARTITIONS, internalSpec.getPartitionCount());
-    assertEquals(changeLogProps, ((KafkaStreamSpec) internalSpec).getProperties());
   }
 
   @Test
-  public void testCreateChangelogStreamDelegatesToCreateStream_specialCharsInTopicName() {
-    final String STREAM = "test.Change_Log.Stream";
+  public void testCreateChangelogStream() {
+    final String STREAM = "testChangeLogStream";
     final int PARTITIONS = 12;
-    final int REP_FACTOR = 3;
+    final int REP_FACTOR = 1;
 
     Properties coordProps = new Properties();
     Properties changeLogProps = new Properties();
@@ -94,60 +90,56 @@ public class TestKafkaSystemAdminJava extends TestKafkaSystemAdmin {
     Map<String, ChangelogInfo> changeLogMap = new HashMap<>();
     changeLogMap.put(STREAM, new ChangelogInfo(REP_FACTOR, changeLogProps));
 
-    SystemAdmin admin = Mockito.spy(createSystemAdmin(coordProps, 3, Util.javaMapAsScalaMap(changeLogMap)));
-    StreamSpec spec = new StreamSpec(KafkaSystemAdmin.CHANGELOG_STREAMID(), STREAM, SYSTEM(), PARTITIONS);
-    admin.createChangelogStream(STREAM, PARTITIONS);
-    admin.validateStream(spec);
+    KafkaSystemAdmin admin = Mockito.spy(createSystemAdmin(coordProps, 1, Util.javaMapAsScalaMap(changeLogMap)));
+    StreamSpec spec = StreamSpec.createChangeLogStreamSpec(STREAM, SYSTEM(), PARTITIONS);
 
-    ArgumentCaptor<StreamSpec> specCaptor = ArgumentCaptor.forClass(StreamSpec.class);
-    Mockito.verify(admin).createStream(specCaptor.capture());
-
-    StreamSpec internalSpec = specCaptor.getValue();
-    assertTrue(internalSpec instanceof KafkaStreamSpec);  // KafkaStreamSpec is used to carry replication factor
-    assertEquals(KafkaSystemAdmin.CHANGELOG_STREAMID(), internalSpec.getId());
-    assertEquals(SYSTEM(), internalSpec.getSystemName());
-    assertEquals(STREAM, internalSpec.getPhysicalName());
-    assertEquals(REP_FACTOR, ((KafkaStreamSpec) internalSpec).getReplicationFactor());
-    assertEquals(PARTITIONS, internalSpec.getPartitionCount());
-    assertEquals(changeLogProps, ((KafkaStreamSpec) internalSpec).getProperties());
-  }
+    Mockito.doAnswer(invocationOnMock -> {
+      StreamSpec internalSpec = (StreamSpec) invocationOnMock.callRealMethod();
+      assertTrue(internalSpec instanceof KafkaStreamSpec);  // KafkaStreamSpec is used to carry replication factor
+      assertTrue(internalSpec.isChangeLogStream());
+      assertEquals(SYSTEM(), internalSpec.getSystemName());
+      assertEquals(STREAM, internalSpec.getPhysicalName());
+      assertEquals(REP_FACTOR, ((KafkaStreamSpec) internalSpec).getReplicationFactor());
+      assertEquals(PARTITIONS, internalSpec.getPartitionCount());
+      assertEquals(changeLogProps, ((KafkaStreamSpec) internalSpec).getProperties());
 
-  @Test
-  public void testValidateChangelogStreamDelegatesToValidateStream() {
-    final String STREAM = "testChangeLogValidate";
-    Properties coordProps = new Properties();
-    Map<String, ChangelogInfo> changeLogMap = new HashMap<>();
-    changeLogMap.put(STREAM, new ChangelogInfo(3, new Properties()));
-
-    KafkaSystemAdmin systemAdmin = createSystemAdmin(coordProps, 3, Util.javaMapAsScalaMap(changeLogMap));
-    SystemAdmin admin = Mockito.spy(systemAdmin);
-    StreamSpec spec = new StreamSpec("testId", STREAM, "testSystem", 12);
+      return internalSpec;
+    }).when(admin).toKafkaSpec(Mockito.any());
 
-    admin.createChangelogStream(spec.getPhysicalName(), spec.getPartitionCount());
+    admin.createStream(spec);
     admin.validateStream(spec);
-    admin.validateChangelogStream(spec.getPhysicalName(), spec.getPartitionCount());
-
-    Mockito.verify(admin).createStream(Mockito.any());
-    Mockito.verify(admin, Mockito.times(3)).validateStream(Mockito.any());
   }
 
   @Test
-  public void testValidateChangelogStreamDelegatesToCreateStream_specialCharsInTopicName() {
-    final String STREAM = "test.Change_Log.Validate";
+  public void testCreateChangelogStreamWithSpecialCharsInTopicName() {
+    final String STREAM = "test.Change_Log.Stream";
+    final int PARTITIONS = 12;
+    final int REP_FACTOR = 1;
+
     Properties coordProps = new Properties();
+    Properties changeLogProps = new Properties();
+    changeLogProps.setProperty("cleanup.policy", "compact");
+    changeLogProps.setProperty("segment.bytes", "139");
     Map<String, ChangelogInfo> changeLogMap = new HashMap<>();
-    changeLogMap.put(STREAM, new ChangelogInfo(3, new Properties()));
-
-    KafkaSystemAdmin systemAdmin = createSystemAdmin(coordProps, 3, Util.javaMapAsScalaMap(changeLogMap));
-    SystemAdmin admin = Mockito.spy(systemAdmin);
-    StreamSpec spec = new StreamSpec("testId", STREAM, "testSystem", 12);
+    changeLogMap.put(STREAM, new ChangelogInfo(REP_FACTOR, changeLogProps));
 
-    admin.createChangelogStream(spec.getPhysicalName(), spec.getPartitionCount());
+    KafkaSystemAdmin admin = Mockito.spy(createSystemAdmin(coordProps, 1, Util.javaMapAsScalaMap(changeLogMap)));
+    StreamSpec spec = StreamSpec.createChangeLogStreamSpec(STREAM, SYSTEM(), PARTITIONS);
+    Mockito.doAnswer(invocationOnMock -> {
+      StreamSpec internalSpec = (StreamSpec) invocationOnMock.callRealMethod();
+      assertTrue(internalSpec instanceof KafkaStreamSpec);  // KafkaStreamSpec is used to carry replication factor
+      assertTrue(internalSpec.isChangeLogStream());
+      assertEquals(SYSTEM(), internalSpec.getSystemName());
+      assertEquals(STREAM, internalSpec.getPhysicalName());
+      assertEquals(REP_FACTOR, ((KafkaStreamSpec) internalSpec).getReplicationFactor());
+      assertEquals(PARTITIONS, internalSpec.getPartitionCount());
+      assertEquals(changeLogProps, ((KafkaStreamSpec) internalSpec).getProperties());
+
+      return internalSpec;
+    }).when(admin).toKafkaSpec(Mockito.any());
+
+    admin.createStream(spec);
     admin.validateStream(spec);
-    admin.validateChangelogStream(STREAM, spec.getPartitionCount()); // Should not throw
-
-    Mockito.verify(admin).createStream(Mockito.any());
-    Mockito.verify(admin, Mockito.times(3)).validateStream(Mockito.any());
   }
 
   @Test
@@ -191,4 +183,17 @@ public class TestKafkaSystemAdminJava extends TestKafkaSystemAdmin {
 
     admin.validateStream(spec2);
   }
+
+  @Test
+  public void testClearStream() {
+    KafkaSystemAdmin admin = this.basicSystemAdmin;
+    StreamSpec spec = new StreamSpec("testId", "testStreamClear", "testSystem", 8);
+
+    assertTrue("createStream should return true if the stream does not exist and then is created.", admin.createStream(spec));
+    assertTrue(admin.clearStream(spec));
+
+    scala.collection.immutable.Set<String> topic = new scala.collection.immutable.Set.Set1<>(spec.getPhysicalName());
+    scala.collection.immutable.Map<String, TopicMetadata> metadata = admin.getTopicMetadata(topic);
+    assertTrue(metadata.get(spec.getPhysicalName()).get().partitionsMetadata().isEmpty());
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/79200c73/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemAdmin.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemAdmin.scala b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemAdmin.scala
index 19f3903..6fb03a1 100644
--- a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemAdmin.scala
+++ b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemAdmin.scala
@@ -34,7 +34,7 @@ import org.apache.kafka.common.security.JaasUtils
 import org.apache.samza.Partition
 import org.apache.samza.config.KafkaProducerConfig
 import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata
-import org.apache.samza.system.{SystemStreamMetadata, SystemStreamPartition}
+import org.apache.samza.system.{StreamSpec, SystemStreamMetadata, SystemStreamPartition}
 import org.apache.samza.util.{ClientUtilTopicMetadataStore, ExponentialSleepStrategy, KafkaUtil, TopicMetadataStore}
 import org.junit.Assert._
 import org.junit._
@@ -283,7 +283,8 @@ class TestKafkaSystemAdmin {
     val topic = "test-coordinator-stream"
     val systemAdmin = new KafkaSystemAdmin(SYSTEM, brokers, () => ZkUtils(zkConnect, 6000, 6000, zkSecure), coordinatorStreamReplicationFactor = 3)
 
-    systemAdmin.createCoordinatorStream(topic)
+    val spec = StreamSpec.createCoordinatorStreamSpec(topic, "kafka")
+    systemAdmin.createStream(spec)
     validateTopic(topic, 1)
     val topicMetadataMap = TopicMetadataCache.getTopicMetadata(Set(topic), "kafka", metadataStore.getTopicInfo)
     assertTrue(topicMetadataMap.contains(topic))

http://git-wip-us.apache.org/repos/asf/samza/blob/79200c73/samza-test/src/main/java/org/apache/samza/system/mock/MockSystemAdmin.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/system/mock/MockSystemAdmin.java b/samza-test/src/main/java/org/apache/samza/system/mock/MockSystemAdmin.java
index a05f89a..322b367 100644
--- a/samza-test/src/main/java/org/apache/samza/system/mock/MockSystemAdmin.java
+++ b/samza-test/src/main/java/org/apache/samza/system/mock/MockSystemAdmin.java
@@ -66,21 +66,6 @@ public class MockSystemAdmin implements SystemAdmin {
   }
 
   @Override
-  public void createChangelogStream(String streamName, int numOfPartitions) {
-    throw new UnsupportedOperationException("Method not implemented");
-  }
-
-  @Override
-  public void validateChangelogStream(String streamName, int numOfPartitions) {
-    throw new UnsupportedOperationException("Method not implemented");
-  }
-
-  @Override
-  public void createCoordinatorStream(String streamName) {
-    throw new UnsupportedOperationException("Method not implemented.");
-  }
-
-  @Override
   public Integer offsetComparator(String offset1, String offset2) {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/79200c73/samza-test/src/test/java/org/apache/samza/test/util/SimpleSystemAdmin.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/util/SimpleSystemAdmin.java b/samza-test/src/test/java/org/apache/samza/test/util/SimpleSystemAdmin.java
index 41f01c5..8890a2f 100644
--- a/samza-test/src/test/java/org/apache/samza/test/util/SimpleSystemAdmin.java
+++ b/samza-test/src/test/java/org/apache/samza/test/util/SimpleSystemAdmin.java
@@ -63,21 +63,6 @@ public class SimpleSystemAdmin implements SystemAdmin {
   }
 
   @Override
-  public void createChangelogStream(String streamName, int numOfPartitions) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void validateChangelogStream(String streamName, int numOfPartitions) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void createCoordinatorStream(String streamName) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
   public Integer offsetComparator(String offset1, String offset2) {
     if (offset1 == null) {
       return offset2 == null ? 0 : -1;

http://git-wip-us.apache.org/repos/asf/samza/blob/79200c73/samza-yarn/src/test/scala/org/apache/samza/job/yarn/MockSystemAdmin.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/test/scala/org/apache/samza/job/yarn/MockSystemAdmin.scala b/samza-yarn/src/test/scala/org/apache/samza/job/yarn/MockSystemAdmin.scala
index c320a97..5650d4b 100644
--- a/samza-yarn/src/test/scala/org/apache/samza/job/yarn/MockSystemAdmin.scala
+++ b/samza-yarn/src/test/scala/org/apache/samza/job/yarn/MockSystemAdmin.scala
@@ -38,17 +38,5 @@ class MockSystemAdmin(numTasks: Int) extends SystemAdmin {
     }).toMap.asJava
   }
 
-  override def createChangelogStream(topicName: String, numOfChangeLogPartitions: Int) {
-    new UnsupportedOperationException("Method not implemented.")
-  }
-
-  override def validateChangelogStream(topicName: String, numOfChangeLogPartitions: Int) {
-    new UnsupportedOperationException("Method not implemented.")
-  }
-
-  override def createCoordinatorStream(streamName: String) {
-    new UnsupportedOperationException("Method not implemented.")
-  }
-
   override def offsetComparator(offset1: String, offset2: String) = null
 }


[15/16] samza git commit: Merge branch 'master' into 0.14.0

Posted by xi...@apache.org.
Merge branch 'master' into 0.14.0


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

Branch: refs/heads/master
Commit: 052a0570cf0f1a1020faef7c3695c0e86c6f348c
Parents: a1f0144 f16ba26
Author: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>
Authored: Tue Oct 3 15:09:41 2017 -0700
Committer: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>
Committed: Tue Oct 3 15:09:41 2017 -0700

----------------------------------------------------------------------
 build.gradle                                    |  12 +-
 .../versioned/jobs/configuration-table.html     |  27 +-
 .../java/org/apache/samza/operators/KV.java     |  48 +++
 .../apache/samza/operators/MessageStream.java   |  34 +-
 .../apache/samza/operators/OutputStream.java    |   6 +-
 .../org/apache/samza/operators/StreamGraph.java |  67 ++-
 .../org/apache/samza/serializers/Serde.java     |   6 +-
 .../samza/system/SystemProducerException.java   |  38 ++
 .../ByteBufferSerde.scala                       |  48 +++
 .../ByteSerde.scala                             |  36 ++
 .../DoubleSerde.scala                           |  45 ++
 .../IntegerSerde.scala                          |  45 ++
 .../JsonSerdeV2.scala                           |  91 ++++
 .../org.apache.samza.serializers/KVSerde.scala  |  55 +++
 .../LongSerde.scala                             |  45 ++
 .../NoOpSerde.scala                             |  37 ++
 .../SerializableSerde.scala                     |  67 +++
 .../StringSerde.scala                           |  49 +++
 .../UUIDSerde.scala                             |  47 ++
 .../TestByteBufferSerde.scala                   |  53 +++
 .../TestByteSerde.scala                         |  38 ++
 .../TestDoubleSerde.scala                       |  40 ++
 .../TestIntegerSerde.scala                      |  37 ++
 .../TestJsonSerdeV2.scala                       |  45 ++
 .../TestLongSerde.scala                         |  40 ++
 .../TestSerializableSerde.scala                 |  45 ++
 .../TestStringSerde.scala                       |  37 ++
 .../TestUUIDSerde.scala                         |  53 +++
 .../apache/samza/config/JavaSystemConfig.java   |  14 +-
 .../samza/config/JobCoordinatorConfig.java      |   9 +
 .../coordinator/CoordinationUtilsFactory.java   |  10 -
 .../org/apache/samza/execution/JobNode.java     |  92 +++-
 .../samza/operators/MessageStreamImpl.java      |  25 +-
 .../apache/samza/operators/StreamGraphImpl.java | 127 +++---
 .../samza/operators/impl/InputOperatorImpl.java |  16 +-
 .../samza/operators/impl/OperatorImpl.java      |  36 +-
 .../samza/operators/impl/OperatorImplGraph.java |  27 +-
 .../operators/impl/OutputOperatorImpl.java      |  58 +--
 .../operators/impl/PartitionByOperatorImpl.java | 108 +++++
 .../operators/impl/WindowOperatorImpl.java      |   3 +-
 .../samza/operators/spec/InputOperatorSpec.java |  37 +-
 .../samza/operators/spec/OperatorSpec.java      |  20 +-
 .../samza/operators/spec/OperatorSpecs.java     |  26 +-
 .../operators/spec/OutputOperatorSpec.java      |  11 +-
 .../samza/operators/spec/OutputStreamImpl.java  |  28 +-
 .../operators/spec/PartitionByOperatorSpec.java |  81 ++++
 .../stream/IntermediateMessageStreamImpl.java   |  17 +-
 .../samza/runtime/LocalApplicationRunner.java   |   5 +-
 .../serializers/IntermediateMessageSerde.java   |  39 +-
 .../apache/samza/task/StreamOperatorTask.java   |   4 +-
 .../apache/samza/config/SerializerConfig.scala  |   6 +-
 .../org/apache/samza/config/SystemConfig.scala  |  16 +-
 .../org/apache/samza/config/TaskConfig.scala    |   3 +
 .../apache/samza/container/SamzaContainer.scala |  59 ++-
 .../samza/serializers/ByteBufferSerde.scala     |  48 ---
 .../apache/samza/serializers/ByteSerde.scala    |  36 --
 .../apache/samza/serializers/DoubleSerde.scala  |  45 --
 .../apache/samza/serializers/IntegerSerde.scala |  45 --
 .../apache/samza/serializers/JsonSerde.scala    |  32 +-
 .../apache/samza/serializers/LongSerde.scala    |  45 --
 .../serializers/MetricsSnapshotSerde.scala      |   4 +-
 .../samza/serializers/SerializableSerde.scala   |  67 ---
 .../apache/samza/serializers/StringSerde.scala  |  44 --
 .../apache/samza/serializers/UUIDSerde.scala    |  47 --
 .../main/scala/org/apache/samza/util/Util.scala |  15 +-
 .../samza/config/TestJavaSystemConfig.java      |  35 +-
 .../apache/samza/example/BroadcastExample.java  |  18 +-
 .../samza/example/KeyValueStoreExample.java     |  26 +-
 .../org/apache/samza/example/MergeExample.java  |  20 +-
 .../samza/example/OrderShipmentJoinExample.java |  28 +-
 .../samza/example/PageViewCounterExample.java   |  13 +-
 .../samza/example/RepartitionExample.java       |  22 +-
 .../org/apache/samza/example/WindowExample.java |   8 +-
 .../samza/execution/TestExecutionPlanner.java   |  59 ++-
 .../execution/TestJobGraphJsonGenerator.java    |  35 +-
 .../org/apache/samza/execution/TestJobNode.java | 112 +++++
 .../samza/operators/TestJoinOperator.java       |  21 +-
 .../samza/operators/TestMessageStreamImpl.java  |  69 ++-
 .../samza/operators/TestStreamGraphImpl.java    | 407 ++++++++++++++++--
 .../samza/operators/TestWindowOperator.java     |  17 +-
 .../operators/impl/TestOperatorImplGraph.java   |  88 +++-
 .../operators/impl/TestStreamOperatorImpl.java  |   4 +-
 .../runtime/TestLocalApplicationRunner.java     |  22 +-
 .../apache/samza/config/TestSystemConfig.scala  |  67 +++
 .../samza/container/TestTaskInstance.scala      |  35 ++
 .../samza/serializers/TestByteBufferSerde.scala |  53 ---
 .../samza/serializers/TestByteSerde.scala       |  38 --
 .../samza/serializers/TestDoubleSerde.scala     |  40 --
 .../samza/serializers/TestIntegerSerde.scala    |  37 --
 .../samza/serializers/TestLongSerde.scala       |  40 --
 .../serializers/TestSerializableSerde.scala     |  45 --
 .../samza/serializers/TestStringSerde.scala     |  37 --
 .../samza/serializers/TestUUIDSerde.scala       |  53 ---
 .../scala/org/apache/samza/util/TestUtil.scala  |  13 +
 .../org/apache/samza/config/KafkaConfig.scala   |  34 +-
 .../samza/system/kafka/KafkaSystemFactory.scala |   5 +-
 .../system/kafka/KafkaSystemProducer.scala      | 256 ++++++-----
 .../samza/system/kafka/MockKafkaProducer.java   |  87 ++--
 .../kafka/TestKafkaSystemProducerJava.java      |   2 +-
 .../apache/samza/config/TestKafkaConfig.scala   |  24 +-
 .../system/kafka/TestKafkaSystemProducer.scala  | 427 ++++++++++++++++---
 .../storage/kv/TestKeyValueStorageEngine.scala  |  14 +-
 .../apache/samza/config/Log4jSystemConfig.java  |   2 +-
 .../samza/logging/log4j/StreamAppender.java     |   2 +-
 samza-test/src/main/resources/log4j.xml         |  44 +-
 .../EndOfStreamIntegrationTest.java             |  15 +-
 .../WatermarkIntegrationTest.java               |   8 +-
 .../apache/samza/test/operator/PageView.java    |  37 +-
 .../test/operator/RepartitionWindowApp.java     |  28 +-
 .../samza/test/operator/SessionWindowApp.java   |  23 +-
 .../test/operator/TestRepartitionWindowApp.java |  20 +-
 .../samza/test/operator/TumblingWindowApp.java  |  24 +-
 .../test/processor/IdentityStreamTask.java      |  16 +-
 .../test/processor/TestStreamProcessor.java     | 149 ++++---
 .../processor/TestZkLocalApplicationRunner.java |  36 +-
 .../test/integration/StreamTaskTestUtil.scala   |   1 +
 .../test/integration/TestStatefulTask.scala     |   2 +-
 117 files changed, 3580 insertions(+), 1634 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/docs/learn/documentation/versioned/jobs/configuration-table.html
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/samza-core/src/main/java/org/apache/samza/execution/JobNode.java
----------------------------------------------------------------------
diff --cc samza-core/src/main/java/org/apache/samza/execution/JobNode.java
index a86e019,fea42f2..7ff43ed
--- a/samza-core/src/main/java/org/apache/samza/execution/JobNode.java
+++ b/samza-core/src/main/java/org/apache/samza/execution/JobNode.java
@@@ -123,8 -132,11 +132,11 @@@ public class JobNode 
      configs.put(CONFIG_INTERNAL_EXECUTION_PLAN, executionPlanJson);
  
      // write input/output streams to configs
 -    inEdges.stream().filter(StreamEdge::isIntermediate).forEach(edge -> addStreamConfig(edge, configs));
 +    inEdges.stream().filter(StreamEdge::isIntermediate).forEach(edge -> configs.putAll(edge.generateConfig()));
  
+     // write serialized serde instances and stream serde configs to configs
+     addSerdeConfigs(configs);
+ 
      log.info("Job {} has generated configs {}", jobName, configs);
  
      String configPrefix = String.format(CONFIG_JOB_PREFIX, jobName);

http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
----------------------------------------------------------------------
diff --cc samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
index eefd4eb,e353ac4..0c50630
--- a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
@@@ -18,16 -18,9 +18,13 @@@
   */
  package org.apache.samza.operators.impl;
  
- import java.util.Collection;
- import java.util.Collections;
- import java.util.HashSet;
- import java.util.Set;
+ import org.apache.samza.SamzaException;
  import org.apache.samza.config.Config;
  import org.apache.samza.config.MetricsConfig;
 +import org.apache.samza.container.TaskContextImpl;
 +import org.apache.samza.container.TaskName;
 +import org.apache.samza.operators.functions.WatermarkFunction;
 +import org.apache.samza.system.EndOfStreamMessage;
  import org.apache.samza.metrics.Counter;
  import org.apache.samza.metrics.MetricsRegistry;
  import org.apache.samza.metrics.Timer;
@@@ -39,14 -29,20 +36,23 @@@ import org.apache.samza.task.MessageCol
  import org.apache.samza.task.TaskContext;
  import org.apache.samza.task.TaskCoordinator;
  import org.apache.samza.util.HighResolutionClock;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
  
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.HashSet;
+ import java.util.Set;
+ 
+ 
  /**
   * Abstract base class for all stream operator implementations.
+  *
+  * @param <M> type of the input to this operator
+  * @param <RM> type of the results of applying this operator
   */
  public abstract class OperatorImpl<M, RM> {
 +  private static final Logger LOG = LoggerFactory.getLogger(OperatorImpl.class);
    private static final String METRICS_GROUP = OperatorImpl.class.getName();
  
    private boolean initialized;
@@@ -151,14 -133,9 +169,16 @@@
      long endNs = this.highResClock.nanoTime();
      this.handleMessageNs.update(endNs - startNs);
  
-     results.forEach(rm -> this.registeredOperators.forEach(op -> op.onMessage(rm, collector, coordinator)));
+     results.forEach(rm ->
+         this.registeredOperators.forEach(op ->
 -            op.onMessage(rm, collector, coordinator)));
++            op.onMessage(rm, collector, coordinator)));    
 +
 +    WatermarkFunction watermarkFn = getOperatorSpec().getWatermarkFn();
 +    if (watermarkFn != null) {
 +      // check whether there is new watermark emitted from the user function
 +      Long outputWm = watermarkFn.getOutputWatermark();
 +      propagateWatermark(outputWm, collector, coordinator);
 +    }
    }
  
    /**

http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java
----------------------------------------------------------------------
diff --cc samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java
index 9b747bc,faedfc9..1f86975
--- a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java
@@@ -18,14 -18,9 +18,14 @@@
   */
  package org.apache.samza.operators.impl;
  
 +import com.google.common.collect.HashMultimap;
  import com.google.common.collect.Lists;
 +import com.google.common.collect.Multimap;
 +import java.util.stream.Collectors;
- import org.apache.commons.lang3.tuple.Pair;
  import org.apache.samza.config.Config;
 +import org.apache.samza.container.TaskContextImpl;
 +import org.apache.samza.job.model.JobModel;
+ import org.apache.samza.operators.KV;
  import org.apache.samza.operators.StreamGraphImpl;
  import org.apache.samza.operators.functions.JoinFunction;
  import org.apache.samza.operators.functions.PartialJoinFunction;
@@@ -272,70 -249,4 +275,68 @@@ public class OperatorImplGraph 
        }
      };
    }
 +
 +  private boolean hasIntermediateStreams(StreamGraphImpl streamGraph) {
 +    return !Collections.disjoint(streamGraph.getInputOperators().keySet(), streamGraph.getOutputStreams().keySet());
 +  }
 +
 +  /**
 +   * calculate the task count that produces to each intermediate streams
 +   * @param streamToConsumerTasks input streams to task mapping
 +   * @param intermediateToInputStreams intermediate stream to input streams mapping
 +   * @return mapping from intermediate stream to task count
 +   */
 +  static Map<SystemStream, Integer> getProducerTaskCountForIntermediateStreams(
 +      Multimap<SystemStream, String> streamToConsumerTasks,
 +      Multimap<SystemStream, SystemStream> intermediateToInputStreams) {
 +    Map<SystemStream, Integer> result = new HashMap<>();
 +    intermediateToInputStreams.asMap().entrySet().forEach(entry -> {
 +        result.put(entry.getKey(),
 +            entry.getValue().stream()
 +                .flatMap(systemStream -> streamToConsumerTasks.get(systemStream).stream())
 +                .collect(Collectors.toSet()).size());
 +      });
 +    return result;
 +  }
 +
 +  /**
 +   * calculate the mapping from input streams to consumer tasks
 +   * @param jobModel JobModel object
 +   * @return mapping from input stream to tasks
 +   */
 +  static Multimap<SystemStream, String> getStreamToConsumerTasks(JobModel jobModel) {
 +    Multimap<SystemStream, String> streamToConsumerTasks = HashMultimap.create();
 +    jobModel.getContainers().values().forEach(containerModel -> {
 +        containerModel.getTasks().values().forEach(taskModel -> {
 +            taskModel.getSystemStreamPartitions().forEach(ssp -> {
 +                streamToConsumerTasks.put(ssp.getSystemStream(), taskModel.getTaskName().getTaskName());
 +              });
 +          });
 +      });
 +    return streamToConsumerTasks;
 +  }
 +
 +  /**
 +   * calculate the mapping from output streams to input streams
 +   * @param streamGraph the user {@link StreamGraphImpl} instance
 +   * @return mapping from output streams to input streams
 +   */
 +  static Multimap<SystemStream, SystemStream> getIntermediateToInputStreamsMap(StreamGraphImpl streamGraph) {
 +    Multimap<SystemStream, SystemStream> outputToInputStreams = HashMultimap.create();
 +    streamGraph.getInputOperators().entrySet().stream()
 +        .forEach(
 +            entry -> computeOutputToInput(entry.getKey().toSystemStream(), entry.getValue(), outputToInputStreams));
 +    return outputToInputStreams;
 +  }
 +
 +  private static void computeOutputToInput(SystemStream input, OperatorSpec opSpec,
 +      Multimap<SystemStream, SystemStream> outputToInputStreams) {
-     if (opSpec instanceof OutputOperatorSpec) {
-       OutputOperatorSpec outputOpSpec = (OutputOperatorSpec) opSpec;
-       if (outputOpSpec.getOpCode() == OperatorSpec.OpCode.PARTITION_BY) {
-         outputToInputStreams.put(outputOpSpec.getOutputStream().getStreamSpec().toSystemStream(), input);
-       }
++    if (opSpec instanceof PartitionByOperatorSpec) {
++      PartitionByOperatorSpec spec = (PartitionByOperatorSpec) opSpec;
++      outputToInputStreams.put(spec.getOutputStream().getStreamSpec().toSystemStream(), input);
 +    } else {
 +      Collection<OperatorSpec> nextOperators = opSpec.getRegisteredOperatorSpecs();
 +      nextOperators.forEach(spec -> computeOutputToInput(input, spec, outputToInputStreams));
 +    }
 +  }
  }

http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/samza-core/src/main/java/org/apache/samza/operators/impl/PartitionByOperatorImpl.java
----------------------------------------------------------------------
diff --cc samza-core/src/main/java/org/apache/samza/operators/impl/PartitionByOperatorImpl.java
index 0000000,072b31d..28b8dba
mode 000000,100644..100644
--- a/samza-core/src/main/java/org/apache/samza/operators/impl/PartitionByOperatorImpl.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/PartitionByOperatorImpl.java
@@@ -1,0 -1,82 +1,108 @@@
+ /*
+  * 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.SamzaException;
+ import org.apache.samza.config.Config;
++import org.apache.samza.container.TaskContextImpl;
+ import org.apache.samza.operators.KV;
+ import org.apache.samza.operators.spec.OperatorSpec;
+ import org.apache.samza.operators.spec.OutputStreamImpl;
+ import org.apache.samza.operators.spec.PartitionByOperatorSpec;
++import org.apache.samza.system.ControlMessage;
++import org.apache.samza.system.EndOfStreamMessage;
+ import org.apache.samza.system.OutgoingMessageEnvelope;
++import org.apache.samza.system.StreamMetadataCache;
+ import org.apache.samza.system.SystemStream;
++import org.apache.samza.system.WatermarkMessage;
+ 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.Collections;
+ import java.util.function.Function;
+ 
+ 
+ /**
+  * An operator that sends sends messages to an output {@link SystemStream} for repartitioning them.
+  */
+ class PartitionByOperatorImpl<M, K, V> extends OperatorImpl<M, Void> {
+ 
+   private final PartitionByOperatorSpec<M, K, V> partitionByOpSpec;
+   private final SystemStream systemStream;
+   private final Function<? super M, ? extends K> keyFunction;
+   private final Function<? super M, ? extends V> valueFunction;
++  private final String taskName;
++  private final ControlMessageSender controlMessageSender;
+ 
+   PartitionByOperatorImpl(PartitionByOperatorSpec<M, K, V> partitionByOpSpec, Config config, TaskContext context) {
+     this.partitionByOpSpec = partitionByOpSpec;
+     OutputStreamImpl<KV<K, V>> outputStream = partitionByOpSpec.getOutputStream();
+     if (!outputStream.isKeyedOutput()) {
+       throw new SamzaException("Output stream for repartitioning must be a keyed stream.");
+     }
+     this.systemStream = new SystemStream(
+         outputStream.getStreamSpec().getSystemName(),
+         outputStream.getStreamSpec().getPhysicalName());
+     this.keyFunction = partitionByOpSpec.getKeyFunction();
+     this.valueFunction = partitionByOpSpec.getValueFunction();
++    this.taskName = context.getTaskName().getTaskName();
++    StreamMetadataCache streamMetadataCache = ((TaskContextImpl) context).getStreamMetadataCache();
++    this.controlMessageSender = new ControlMessageSender(streamMetadataCache);
+   }
+ 
+   @Override
+   protected void handleInit(Config config, TaskContext context) {
+   }
+ 
+   @Override
+   public Collection<Void> handleMessage(M message, MessageCollector collector,
+       TaskCoordinator coordinator) {
+     K key = keyFunction.apply(message);
+     V value = valueFunction.apply(message);
+     collector.send(new OutgoingMessageEnvelope(systemStream, null, key, value));
+     return Collections.emptyList();
+   }
+ 
+   @Override
+   protected void handleClose() {
+   }
+ 
+   @Override
+   protected OperatorSpec<M, Void> getOperatorSpec() {
+     return partitionByOpSpec;
+   }
++
++  @Override
++  protected void handleEndOfStream(MessageCollector collector, TaskCoordinator coordinator) {
++    sendControlMessage(new EndOfStreamMessage(taskName), collector);
++  }
++
++  @Override
++  protected Long handleWatermark(long watermark, MessageCollector collector, TaskCoordinator coordinator) {
++    sendControlMessage(new WatermarkMessage(watermark, taskName), collector);
++    return watermark;
++  }
++
++  private void sendControlMessage(ControlMessage message, MessageCollector collector) {
++    SystemStream outputStream = partitionByOpSpec.getOutputStream().getStreamSpec().toSystemStream();
++    controlMessageSender.send(message, outputStream, collector);
++  }
+ }

http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java
----------------------------------------------------------------------
diff --cc samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java
index 773f742,2749245..3c66ee6
--- a/samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java
@@@ -18,12 -18,10 +18,11 @@@
   */
  package org.apache.samza.operators.spec;
  
- import org.apache.commons.lang3.tuple.Pair;
+ import org.apache.samza.operators.KV;
+ import org.apache.samza.serializers.Serde;
 +import org.apache.samza.operators.functions.WatermarkFunction;
  import org.apache.samza.system.StreamSpec;
  
- import java.util.function.BiFunction;
- 
  /**
   * The spec for an operator that receives incoming messages from an input stream
   * and converts them to the input message.
@@@ -47,12 -49,15 +50,20 @@@ public class InputOperatorSpec<K, V> ex
      return this.streamSpec;
    }
  
-   public BiFunction<K, V, M> getMsgBuilder() {
-     return this.msgBuilder;
+   public Serde<K> getKeySerde() {
+     return keySerde;
+   }
+ 
+   public Serde<V> getValueSerde() {
+     return valueSerde;
+   }
+ 
+   public boolean isKeyedInput() {
+     return isKeyedInput;
    }
 +
 +  @Override
 +  public WatermarkFunction getWatermarkFn() {
 +    return null;
-   }
++  }  
  }

http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
----------------------------------------------------------------------
diff --cc samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
index 4047d92,bcb0485..71a9897
--- a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
@@@ -19,7 -19,8 +19,9 @@@
  package org.apache.samza.operators.spec;
  
  import org.apache.samza.annotation.InterfaceStability;
 +import org.apache.samza.operators.functions.WatermarkFunction;
+ import org.apache.samza.operators.MessageStream;
+ import org.apache.samza.operators.MessageStreamImpl;
  
  import java.util.Collection;
  import java.util.LinkedHashSet;

http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/samza-core/src/main/java/org/apache/samza/operators/spec/OutputOperatorSpec.java
----------------------------------------------------------------------
diff --cc samza-core/src/main/java/org/apache/samza/operators/spec/OutputOperatorSpec.java
index 9759392,fc88634..862370f
--- a/samza-core/src/main/java/org/apache/samza/operators/spec/OutputOperatorSpec.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/OutputOperatorSpec.java
@@@ -51,12 -46,7 +48,12 @@@ public class OutputOperatorSpec<M> exte
     * The {@link OutputStreamImpl} that this operator is sending its output to.
     * @return the {@link OutputStreamImpl} for this operator if any, else null.
     */
-   public OutputStreamImpl<?, ?, M> getOutputStream() {
+   public OutputStreamImpl<M> getOutputStream() {
      return this.outputStream;
    }
 +
 +  @Override
 +  public WatermarkFunction getWatermarkFn() {
 +    return null;
 +  }
  }

http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/samza-core/src/main/java/org/apache/samza/operators/spec/PartitionByOperatorSpec.java
----------------------------------------------------------------------
diff --cc samza-core/src/main/java/org/apache/samza/operators/spec/PartitionByOperatorSpec.java
index 0000000,a2bb5f2..42eeb4b
mode 000000,100644..100644
--- a/samza-core/src/main/java/org/apache/samza/operators/spec/PartitionByOperatorSpec.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/PartitionByOperatorSpec.java
@@@ -1,0 -1,76 +1,81 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  *   http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing,
+  * software distributed under the License is distributed on an
+  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  * KIND, either express or implied.  See the License for the
+  * specific language governing permissions and limitations
+  * under the License.
+  */
+ package org.apache.samza.operators.spec;
+ 
+ import org.apache.samza.operators.KV;
++import org.apache.samza.operators.functions.WatermarkFunction;
+ 
+ import java.util.function.Function;
+ 
+ 
+ /**
+  * The spec for an operator that re-partitions a {@link org.apache.samza.operators.MessageStream} to a
+  * {@link org.apache.samza.system.SystemStream}. This is usually paired with a corresponding
+  * {@link InputOperatorSpec} that consumes the {@link org.apache.samza.system.SystemStream} again.
+  * <p>
+  * This is a terminal operator and does not allow further operator chaining.
+  *
+  * @param <M> the type of message
+  * @param <K> the type of key in the message
+  * @param <V> the type of value in the message
+  */
+ public class PartitionByOperatorSpec<M, K, V> extends OperatorSpec<M, Void> {
+ 
+   private final OutputStreamImpl<KV<K, V>> outputStream;
+   private final Function<? super M, ? extends K> keyFunction;
+   private final Function<? super M, ? extends V> valueFunction;
+ 
+   /**
+    * Constructs an {@link PartitionByOperatorSpec} to send messages to the provided {@code outputStream}
+    *
+    * @param outputStream the {@link OutputStreamImpl} to send messages to
+    * @param keyFunction the {@link Function} for extracting the key from the message
+    * @param valueFunction the {@link Function} for extracting the value from the message
+    * @param opId the unique ID of this {@link SinkOperatorSpec} in the graph
+    */
+   PartitionByOperatorSpec(OutputStreamImpl<KV<K, V>> outputStream,
+       Function<? super M, ? extends K> keyFunction,
+       Function<? super M, ? extends V> valueFunction, int opId) {
+     super(OpCode.PARTITION_BY, opId);
+     this.outputStream = outputStream;
+     this.keyFunction = keyFunction;
+     this.valueFunction = valueFunction;
+   }
+ 
+   /**
+    * The {@link OutputStreamImpl} that this operator is sending its output to.
+    * @return the {@link OutputStreamImpl} for this operator if any, else null.
+    */
+   public OutputStreamImpl<KV<K, V>> getOutputStream() {
+     return this.outputStream;
+   }
+ 
+   public Function<? super M, ? extends K> getKeyFunction() {
+     return keyFunction;
+   }
+ 
+   public Function<? super M, ? extends V> getValueFunction() {
+     return valueFunction;
+   }
+ 
++  @Override
++  public WatermarkFunction getWatermarkFn() {
++    return null;
++  }
+ }

http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/samza-core/src/main/java/org/apache/samza/serializers/IntermediateMessageSerde.java
----------------------------------------------------------------------
diff --cc samza-core/src/main/java/org/apache/samza/serializers/IntermediateMessageSerde.java
index 2ed559f,45ce9aa..61735ad
--- a/samza-core/src/main/java/org/apache/samza/serializers/IntermediateMessageSerde.java
+++ b/samza-core/src/main/java/org/apache/samza/serializers/IntermediateMessageSerde.java
@@@ -20,11 -20,13 +20,13 @@@
  package org.apache.samza.serializers;
  
  import java.util.Arrays;
+ 
  import org.apache.samza.SamzaException;
 -import org.apache.samza.message.EndOfStreamMessage;
 -import org.apache.samza.message.MessageType;
 -import org.apache.samza.message.WatermarkMessage;
 +import org.apache.samza.system.EndOfStreamMessage;
 +import org.apache.samza.system.MessageType;
 +import org.apache.samza.system.WatermarkMessage;
- import org.codehaus.jackson.type.TypeReference;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
  
  
  /**

http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java
----------------------------------------------------------------------
diff --cc samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java
index 0074e24,d7c2742..87af392
--- a/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java
+++ b/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java
@@@ -18,12 -18,10 +18,12 @@@
   */
  package org.apache.samza.task;
  
- import org.apache.commons.lang3.tuple.Pair;
  import org.apache.samza.application.StreamApplication;
  import org.apache.samza.config.Config;
 +import org.apache.samza.system.EndOfStreamMessage;
 +import org.apache.samza.system.MessageType;
  import org.apache.samza.operators.ContextManager;
+ import org.apache.samza.operators.KV;
  import org.apache.samza.operators.StreamGraphImpl;
  import org.apache.samza.operators.impl.InputOperatorImpl;
  import org.apache.samza.operators.impl.OperatorImplGraph;
@@@ -111,21 -105,7 +111,21 @@@ public final class StreamOperatorTask i
      SystemStream systemStream = ime.getSystemStreamPartition().getSystemStream();
      InputOperatorImpl inputOpImpl = operatorImplGraph.getInputOperator(systemStream);
      if (inputOpImpl != null) {
 -      inputOpImpl.onMessage(KV.of(ime.getKey(), ime.getMessage()), collector, coordinator);
 +      switch (MessageType.of(ime.getMessage())) {
 +        case USER_MESSAGE:
-           inputOpImpl.onMessage(Pair.of(ime.getKey(), ime.getMessage()), collector, coordinator);
++          inputOpImpl.onMessage(KV.of(ime.getKey(), ime.getMessage()), collector, coordinator);
 +          break;
 +
 +        case END_OF_STREAM:
 +          EndOfStreamMessage eosMessage = (EndOfStreamMessage) ime.getMessage();
 +          inputOpImpl.aggregateEndOfStream(eosMessage, ime.getSystemStreamPartition(), collector, coordinator);
 +          break;
 +
 +        case WATERMARK:
 +          WatermarkMessage watermarkMessage = (WatermarkMessage) ime.getMessage();
 +          inputOpImpl.aggregateWatermark(watermarkMessage, ime.getSystemStreamPartition(), collector, coordinator);
 +          break;
 +      }
      }
    }
  

http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/samza-core/src/test/java/org/apache/samza/execution/TestExecutionPlanner.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java
----------------------------------------------------------------------
diff --cc samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java
index 0000000,c59c0cc..918da26
mode 000000,100644..100644
--- a/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java
+++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java
@@@ -1,0 -1,111 +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.execution;
+ 
+ import org.apache.samza.config.Config;
+ import org.apache.samza.config.MapConfig;
+ import org.apache.samza.config.SerializerConfig;
+ import org.apache.samza.operators.KV;
+ import org.apache.samza.operators.MessageStream;
+ import org.apache.samza.operators.OutputStream;
+ import org.apache.samza.operators.StreamGraphImpl;
+ import org.apache.samza.runtime.ApplicationRunner;
+ import org.apache.samza.serializers.JsonSerdeV2;
+ import org.apache.samza.serializers.KVSerde;
+ import org.apache.samza.serializers.Serde;
+ import org.apache.samza.serializers.SerializableSerde;
+ import org.apache.samza.serializers.StringSerde;
+ import org.apache.samza.system.StreamSpec;
+ import org.junit.Test;
+ 
+ import java.util.Base64;
+ import java.util.HashMap;
+ import java.util.Map;
+ import java.util.stream.Collectors;
+ 
+ import static org.junit.Assert.assertEquals;
+ import static org.junit.Assert.assertTrue;
+ import static org.mockito.Mockito.doReturn;
+ import static org.mockito.Mockito.mock;
+ 
+ public class TestJobNode {
+ 
+   @Test
+   public void testAddSerdeConfigs() {
+     ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+     StreamSpec inputSpec = new StreamSpec("input", "input", "input-system");
+     StreamSpec outputSpec = new StreamSpec("output", "output", "output-system");
+     StreamSpec partitionBySpec = new StreamSpec("null-null-partition_by-1", "partition_by-1", "intermediate-system");
+     doReturn(inputSpec).when(mockRunner).getStreamSpec("input");
+     doReturn(outputSpec).when(mockRunner).getStreamSpec("output");
+     doReturn(partitionBySpec).when(mockRunner).getStreamSpec("null-null-partition_by-1");
+ 
+     StreamGraphImpl streamGraph = new StreamGraphImpl(mockRunner, mock(Config.class));
+     streamGraph.setDefaultSerde(KVSerde.of(new StringSerde(), new JsonSerdeV2<>()));
+     MessageStream<KV<String, Object>> input = streamGraph.getInputStream("input");
+     OutputStream<KV<String, Object>> output = streamGraph.getOutputStream("output");
+     input.partitionBy(KV::getKey, KV::getValue).sendTo(output);
+ 
+     JobNode jobNode = new JobNode("jobName", "jobId", streamGraph, mock(Config.class));
 -    StreamEdge inputEdge = new StreamEdge(inputSpec);
 -    StreamEdge outputEdge = new StreamEdge(outputSpec);
 -    StreamEdge repartitionEdge = new StreamEdge(partitionBySpec, true);
++    Config config = new MapConfig();
++    StreamEdge inputEdge = new StreamEdge(inputSpec, config);
++    StreamEdge outputEdge = new StreamEdge(outputSpec, config);
++    StreamEdge repartitionEdge = new StreamEdge(partitionBySpec, true, config);
+     jobNode.addInEdge(inputEdge);
+     jobNode.addOutEdge(outputEdge);
+     jobNode.addInEdge(repartitionEdge);
+     jobNode.addOutEdge(repartitionEdge);
+ 
+     Map<String, String> configs = new HashMap<>();
+     jobNode.addSerdeConfigs(configs);
+ 
+     MapConfig mapConfig = new MapConfig(configs);
+     Config serializers = mapConfig.subset("serializers.registry.", true);
+ 
+     // make sure that the serializers deserialize correctly
+     SerializableSerde<Serde> serializableSerde = new SerializableSerde<>();
+     Map<String, Serde> deserializedSerdes = serializers.entrySet().stream().collect(Collectors.toMap(
+         e -> e.getKey().replace(SerializerConfig.SERIALIZED_INSTANCE_SUFFIX(), ""),
+         e -> serializableSerde.fromBytes(Base64.getDecoder().decode(e.getValue().getBytes()))
+     ));
+     assertEquals(2, serializers.size());
+ 
+     String inputKeySerde = mapConfig.get("streams.input.samza.key.serde");
+     String inputMsgSerde = mapConfig.get("streams.input.samza.msg.serde");
+     assertTrue(deserializedSerdes.containsKey(inputKeySerde));
+     assertTrue(inputKeySerde.startsWith(StringSerde.class.getSimpleName()));
+     assertTrue(deserializedSerdes.containsKey(inputMsgSerde));
+     assertTrue(inputMsgSerde.startsWith(JsonSerdeV2.class.getSimpleName()));
+ 
+     String outputKeySerde = mapConfig.get("streams.output.samza.key.serde");
+     String outputMsgSerde = mapConfig.get("streams.output.samza.msg.serde");
+     assertTrue(deserializedSerdes.containsKey(outputKeySerde));
+     assertTrue(outputKeySerde.startsWith(StringSerde.class.getSimpleName()));
+     assertTrue(deserializedSerdes.containsKey(outputMsgSerde));
+     assertTrue(outputMsgSerde.startsWith(JsonSerdeV2.class.getSimpleName()));
+ 
+     String partitionByKeySerde = mapConfig.get("streams.null-null-partition_by-1.samza.key.serde");
+     String partitionByMsgSerde = mapConfig.get("streams.null-null-partition_by-1.samza.msg.serde");
+     assertTrue(deserializedSerdes.containsKey(partitionByKeySerde));
+     assertTrue(partitionByKeySerde.startsWith(StringSerde.class.getSimpleName()));
+     assertTrue(deserializedSerdes.containsKey(partitionByMsgSerde));
+     assertTrue(partitionByMsgSerde.startsWith(JsonSerdeV2.class.getSimpleName()));
+   }
+ 
+ }

http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/samza-core/src/test/java/org/apache/samza/operators/TestJoinOperator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/samza-core/src/test/java/org/apache/samza/operators/TestWindowOperator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java
----------------------------------------------------------------------
diff --cc samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java
index 9fab1b7,68b4ce0..d73c545
--- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java
+++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java
@@@ -19,24 -19,9 +19,24 @@@
  
  package org.apache.samza.operators.impl;
  
 +import com.google.common.collect.HashMultimap;
 +import com.google.common.collect.Multimap;
 +import java.util.Collection;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.Map;
 +import java.util.Set;
- import org.apache.commons.lang3.tuple.Pair;
 +import org.apache.samza.Partition;
  import org.apache.samza.config.Config;
 +import org.apache.samza.config.JobConfig;
 +import org.apache.samza.config.MapConfig;
 +import org.apache.samza.container.TaskContextImpl;
 +import org.apache.samza.container.TaskName;
 +import org.apache.samza.job.model.ContainerModel;
 +import org.apache.samza.job.model.JobModel;
 +import org.apache.samza.job.model.TaskModel;
  import org.apache.samza.metrics.MetricsRegistryMap;
+ import org.apache.samza.operators.KV;
  import org.apache.samza.operators.MessageStream;
  import org.apache.samza.operators.OutputStream;
  import org.apache.samza.operators.StreamGraphImpl;
@@@ -45,9 -30,12 +45,14 @@@ import org.apache.samza.operators.funct
  import org.apache.samza.operators.functions.MapFunction;
  import org.apache.samza.operators.spec.OperatorSpec.OpCode;
  import org.apache.samza.runtime.ApplicationRunner;
+ import org.apache.samza.serializers.IntegerSerde;
+ import org.apache.samza.serializers.KVSerde;
+ import org.apache.samza.serializers.NoOpSerde;
++import org.apache.samza.serializers.Serde;
+ import org.apache.samza.serializers.StringSerde;
  import org.apache.samza.system.StreamSpec;
  import org.apache.samza.system.SystemStream;
 +import org.apache.samza.system.SystemStreamPartition;
  import org.apache.samza.task.MessageCollector;
  import org.apache.samza.task.TaskContext;
  import org.apache.samza.task.TaskCoordinator;
@@@ -121,6 -105,43 +123,47 @@@ public class TestOperatorImplGraph 
    }
  
    @Test
+   public void testPartitionByChain() {
+     ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+     when(mockRunner.getStreamSpec(eq("input"))).thenReturn(new StreamSpec("input", "input-stream", "input-system"));
+     when(mockRunner.getStreamSpec(eq("output"))).thenReturn(new StreamSpec("output", "output-stream", "output-system"));
+     when(mockRunner.getStreamSpec(eq("null-null-partition_by-1")))
+         .thenReturn(new StreamSpec("intermediate", "intermediate-stream", "intermediate-system"));
+     StreamGraphImpl streamGraph = new StreamGraphImpl(mockRunner, mock(Config.class));
+     MessageStream<Object> inputStream = streamGraph.getInputStream("input");
+     OutputStream<KV<Integer, String>> outputStream = streamGraph
+         .getOutputStream("output", KVSerde.of(mock(IntegerSerde.class), mock(StringSerde.class)));
+ 
+     inputStream
+         .partitionBy(Object::hashCode, Object::toString, KVSerde.of(mock(IntegerSerde.class), mock(StringSerde.class)))
+         .sendTo(outputStream);
+ 
 -    TaskContext mockTaskContext = mock(TaskContext.class);
++    TaskContextImpl mockTaskContext = mock(TaskContextImpl.class);
+     when(mockTaskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap());
++    when(mockTaskContext.getTaskName()).thenReturn(new TaskName("task 0"));
++    JobModel jobModel = mock(JobModel.class);
++    when(jobModel.getContainers()).thenReturn(Collections.EMPTY_MAP);
++    when(mockTaskContext.getJobModel()).thenReturn(jobModel);
+     OperatorImplGraph opImplGraph =
+         new OperatorImplGraph(streamGraph, mock(Config.class), mockTaskContext, mock(Clock.class));
+ 
+     InputOperatorImpl inputOpImpl = opImplGraph.getInputOperator(new SystemStream("input-system", "input-stream"));
+     assertEquals(1, inputOpImpl.registeredOperators.size());
+ 
+     OperatorImpl partitionByOpImpl = (PartitionByOperatorImpl) inputOpImpl.registeredOperators.iterator().next();
+     assertEquals(0, partitionByOpImpl.registeredOperators.size()); // is terminal but paired with an input operator
+     assertEquals(OpCode.PARTITION_BY, partitionByOpImpl.getOperatorSpec().getOpCode());
+ 
+     InputOperatorImpl repartitionedInputOpImpl =
+         opImplGraph.getInputOperator(new SystemStream("intermediate-system", "intermediate-stream"));
+     assertEquals(1, repartitionedInputOpImpl.registeredOperators.size());
+ 
+     OperatorImpl sendToOpImpl = (OutputOperatorImpl) repartitionedInputOpImpl.registeredOperators.iterator().next();
+     assertEquals(0, sendToOpImpl.registeredOperators.size());
+     assertEquals(OpCode.SEND_TO, sendToOpImpl.getOperatorSpec().getOpCode());
+   }
+ 
+   @Test
    public void testBroadcastChain() {
      ApplicationRunner mockRunner = mock(ApplicationRunner.class);
      when(mockRunner.getStreamSpec(eq("input"))).thenReturn(new StreamSpec("input", "input-stream", "input-system"));
@@@ -173,11 -194,11 +216,11 @@@
      StreamGraphImpl streamGraph = new StreamGraphImpl(mockRunner, mock(Config.class));
  
      JoinFunction mockJoinFunction = mock(JoinFunction.class);
-     MessageStream<Object> inputStream1 = streamGraph.getInputStream("input1", (k, v) -> v);
-     MessageStream<Object> inputStream2 = streamGraph.getInputStream("input2", (k, v) -> v);
+     MessageStream<Object> inputStream1 = streamGraph.getInputStream("input1", new NoOpSerde<>());
+     MessageStream<Object> inputStream2 = streamGraph.getInputStream("input2", new NoOpSerde<>());
      inputStream1.join(inputStream2, mockJoinFunction, Duration.ofHours(1));
  
 -    TaskContext mockTaskContext = mock(TaskContext.class);
 +    TaskContextImpl mockTaskContext = mock(TaskContextImpl.class);
      when(mockTaskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap());
      OperatorImplGraph opImplGraph =
          new OperatorImplGraph(streamGraph, mock(Config.class), mockTaskContext, mock(Clock.class));
@@@ -272,144 -293,4 +315,143 @@@
        }
      };
    }
 +
 +  @Test
 +  public void testGetStreamToConsumerTasks() {
 +    String system = "test-system";
 +    String stream0 = "test-stream-0";
 +    String stream1 = "test-stream-1";
 +
 +    SystemStreamPartition ssp0 = new SystemStreamPartition(system, stream0, new Partition(0));
 +    SystemStreamPartition ssp1 = new SystemStreamPartition(system, stream0, new Partition(1));
 +    SystemStreamPartition ssp2 = new SystemStreamPartition(system, stream1, new Partition(0));
 +
 +    TaskName task0 = new TaskName("Task 0");
 +    TaskName task1 = new TaskName("Task 1");
 +    Set<SystemStreamPartition> ssps = new HashSet<>();
 +    ssps.add(ssp0);
 +    ssps.add(ssp2);
 +    TaskModel tm0 = new TaskModel(task0, ssps, new Partition(0));
 +    ContainerModel cm0 = new ContainerModel("c0", 0, Collections.singletonMap(task0, tm0));
 +    TaskModel tm1 = new TaskModel(task1, Collections.singleton(ssp1), new Partition(1));
 +    ContainerModel cm1 = new ContainerModel("c1", 1, Collections.singletonMap(task1, tm1));
 +
 +    Map<String, ContainerModel> cms = new HashMap<>();
 +    cms.put(cm0.getProcessorId(), cm0);
 +    cms.put(cm1.getProcessorId(), cm1);
 +
 +    JobModel jobModel = new JobModel(new MapConfig(), cms, null);
 +    Multimap<SystemStream, String> streamToTasks = OperatorImplGraph.getStreamToConsumerTasks(jobModel);
 +    assertEquals(streamToTasks.get(ssp0.getSystemStream()).size(), 2);
 +    assertEquals(streamToTasks.get(ssp2.getSystemStream()).size(), 1);
 +  }
 +
 +  @Test
 +  public void testGetOutputToInputStreams() {
 +    Map<String, String> configMap = new HashMap<>();
 +    configMap.put(JobConfig.JOB_NAME(), "test-app");
 +    configMap.put(JobConfig.JOB_DEFAULT_SYSTEM(), "test-system");
 +    Config config = new MapConfig(configMap);
 +
 +    /**
 +     * the graph looks like the following. number of partitions in parentheses. quotes indicate expected value.
 +     *
 +     *                                    input1 -> map -> join -> partitionBy (10) -> output1
 +     *                                                       |
 +     *                                     input2 -> filter -|
 +     *                                                       |
 +     *           input3 -> filter -> partitionBy -> map -> join -> output2
 +     *
 +     */
 +    StreamSpec input1 = new StreamSpec("input1", "input1", "system1");
 +    StreamSpec input2 = new StreamSpec("input2", "input2", "system2");
 +    StreamSpec input3 = new StreamSpec("input3", "input3", "system2");
 +
 +    StreamSpec output1 = new StreamSpec("output1", "output1", "system1");
 +    StreamSpec output2 = new StreamSpec("output2", "output2", "system2");
 +
 +    ApplicationRunner runner = mock(ApplicationRunner.class);
 +    when(runner.getStreamSpec("input1")).thenReturn(input1);
 +    when(runner.getStreamSpec("input2")).thenReturn(input2);
 +    when(runner.getStreamSpec("input3")).thenReturn(input3);
 +    when(runner.getStreamSpec("output1")).thenReturn(output1);
 +    when(runner.getStreamSpec("output2")).thenReturn(output2);
 +
 +    // intermediate streams used in tests
 +    StreamSpec int1 = new StreamSpec("test-app-1-partition_by-10", "test-app-1-partition_by-10", "default-system");
 +    StreamSpec int2 = new StreamSpec("test-app-1-partition_by-6", "test-app-1-partition_by-6", "default-system");
 +    when(runner.getStreamSpec("test-app-1-partition_by-10"))
 +        .thenReturn(int1);
 +    when(runner.getStreamSpec("test-app-1-partition_by-6"))
 +        .thenReturn(int2);
 +
 +    StreamGraphImpl streamGraph = new StreamGraphImpl(runner, config);
-     BiFunction msgBuilder = mock(BiFunction.class);
-     MessageStream m1 = streamGraph.getInputStream("input1", msgBuilder).map(m -> m);
-     MessageStream m2 = streamGraph.getInputStream("input2", msgBuilder).filter(m -> true);
-     MessageStream m3 = streamGraph.getInputStream("input3", msgBuilder).filter(m -> true).partitionBy(m -> "hehe").map(m -> m);
-     Function mockFn = mock(Function.class);
-     OutputStream<Object, Object, Object> om1 = streamGraph.getOutputStream("output1", mockFn, mockFn);
-     OutputStream<Object, Object, Object> om2 = streamGraph.getOutputStream("output2", mockFn, mockFn);
- 
-     m1.join(m2, mock(JoinFunction.class), Duration.ofHours(2)).partitionBy(m -> "haha").sendTo(om1);
++    Serde inputSerde = new NoOpSerde<>();
++    MessageStream m1 = streamGraph.getInputStream("input1", inputSerde).map(m -> m);
++    MessageStream m2 = streamGraph.getInputStream("input2", inputSerde).filter(m -> true);
++    MessageStream m3 = streamGraph.getInputStream("input3", inputSerde).filter(m -> true).partitionBy(m -> "hehe", m -> m).map(m -> m);
++    OutputStream<Object> om1 = streamGraph.getOutputStream("output1");
++    OutputStream<Object> om2 = streamGraph.getOutputStream("output2");
++
++    m1.join(m2, mock(JoinFunction.class), Duration.ofHours(2)).partitionBy(m -> "haha", m -> m).sendTo(om1);
 +    m3.join(m2, mock(JoinFunction.class), Duration.ofHours(1)).sendTo(om2);
 +
 +    Multimap<SystemStream, SystemStream> outputToInput = OperatorImplGraph.getIntermediateToInputStreamsMap(streamGraph);
 +    Collection<SystemStream> inputs = outputToInput.get(int1.toSystemStream());
 +    assertEquals(inputs.size(), 2);
 +    assertTrue(inputs.contains(input1.toSystemStream()));
 +    assertTrue(inputs.contains(input2.toSystemStream()));
 +
 +    inputs = outputToInput.get(int2.toSystemStream());
 +    assertEquals(inputs.size(), 1);
 +    assertEquals(inputs.iterator().next(), input3.toSystemStream());
 +  }
 +
 +  @Test
 +  public void testGetProducerTaskCountForIntermediateStreams() {
 +    /**
 +     * the task assignment looks like the following:
 +     *
 +     * input1 -----> task0, task1 -----> int1
 +     *                                    ^
 +     * input2 ------> task1, task2--------|
 +     *                                    v
 +     * input3 ------> task1 -----------> int2
 +     *
 +     */
 +
 +    SystemStream input1 = new SystemStream("system1", "intput1");
 +    SystemStream input2 = new SystemStream("system2", "intput2");
 +    SystemStream input3 = new SystemStream("system2", "intput3");
 +
 +    SystemStream int1 = new SystemStream("system1", "int1");
 +    SystemStream int2 = new SystemStream("system1", "int2");
 +
 +
 +    String task0 = "Task 0";
 +    String task1 = "Task 1";
 +    String task2 = "Task 2";
 +
 +    Multimap<SystemStream, String> streamToConsumerTasks = HashMultimap.create();
 +    streamToConsumerTasks.put(input1, task0);
 +    streamToConsumerTasks.put(input1, task1);
 +    streamToConsumerTasks.put(input2, task1);
 +    streamToConsumerTasks.put(input2, task2);
 +    streamToConsumerTasks.put(input3, task1);
 +    streamToConsumerTasks.put(int1, task0);
 +    streamToConsumerTasks.put(int1, task1);
 +    streamToConsumerTasks.put(int2, task0);
 +
 +    Multimap<SystemStream, SystemStream> intermediateToInputStreams = HashMultimap.create();
 +    intermediateToInputStreams.put(int1, input1);
 +    intermediateToInputStreams.put(int1, input2);
 +
 +    intermediateToInputStreams.put(int2, input2);
 +    intermediateToInputStreams.put(int2, input3);
 +
 +    Map<SystemStream, Integer> counts = OperatorImplGraph.getProducerTaskCountForIntermediateStreams(
 +        streamToConsumerTasks, intermediateToInputStreams);
 +    assertTrue(counts.get(int1) == 3);
 +    assertTrue(counts.get(int2) == 2);
 +  }
  }

http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
----------------------------------------------------------------------
diff --cc samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
index d2094b4,1816380..a23e513
--- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
+++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
@@@ -20,9 -20,12 +20,12 @@@
  package org.apache.samza.runtime;
  
  import com.google.common.collect.ImmutableList;
- import java.util.*;
 -import java.lang.reflect.Field;
+ import java.util.Collections;
+ import java.util.HashMap;
+ import java.util.List;
+ import java.util.Map;
++import java.util.Set;
  import java.util.stream.Collectors;
- 
  import org.apache.samza.application.StreamApplication;
  import org.apache.samza.config.ApplicationConfig;
  import org.apache.samza.config.JobConfig;
@@@ -49,8 -55,6 +54,7 @@@ import static org.junit.Assert.assertNo
  import static org.mockito.Matchers.anyObject;
  import static org.mockito.Matchers.anyString;
  import static org.mockito.Mockito.*;
 +import static org.powermock.api.mockito.PowerMockito.doReturn;
- import static org.powermock.api.mockito.PowerMockito.mockStatic;
  
  
  @RunWith(PowerMockRunner.class)
@@@ -72,20 -76,44 +76,21 @@@ public class TestLocalApplicationRunne
      StreamApplication app = mock(StreamApplication.class);
      doNothing().when(app).init(anyObject(), anyObject());
  
 -    ExecutionPlanner planner = mock(ExecutionPlanner.class);
 -    Field plannerField = runner.getClass().getSuperclass().getDeclaredField("planner");
 -    plannerField.setAccessible(true);
 -    plannerField.set(runner, planner);
 -
      StreamManager streamManager = mock(StreamManager.class);
 -    Field streamManagerField = runner.getClass().getSuperclass().getDeclaredField("streamManager");
 -    streamManagerField.setAccessible(true);
 -    streamManagerField.set(runner, streamManager);
 -    ArgumentCaptor<List> captor = ArgumentCaptor.forClass(List.class);
 +    doReturn(streamManager).when(runner).getStreamManager();
  
 -    ExecutionPlan plan = new ExecutionPlan() {
 -      @Override
 -      public List<JobConfig> getJobConfigs() {
 -        return Collections.emptyList();
 -      }
 -
 -      @Override
 -      public List<StreamSpec> getIntermediateStreams() {
 -        return Collections.singletonList(new StreamSpec("test-stream", "test-stream", "test-system"));
 -      }
 -
 -      @Override
 -      public String getPlanAsJson()
 -          throws Exception {
 -        return "";
 -      }
 -    };
 -    when(planner.plan(anyObject())).thenReturn(plan);
 +    ExecutionPlan plan = mock(ExecutionPlan.class);
 +    when(plan.getIntermediateStreams()).thenReturn(Collections.singletonList(new StreamSpec("test-stream", "test-stream", "test-system")));
 +    when(plan.getPlanAsJson()).thenReturn("");
 +    doReturn(plan).when(runner).getExecutionPlan(any(), any());
  
-     mockStatic(CoordinationUtilsFactory.class);
      CoordinationUtilsFactory coordinationUtilsFactory = mock(CoordinationUtilsFactory.class);
-     when(CoordinationUtilsFactory.getCoordinationUtilsFactory(anyObject())).thenReturn(coordinationUtilsFactory);
+     JobCoordinatorConfig mockJcConfig = mock(JobCoordinatorConfig.class);
+     when(mockJcConfig.getCoordinationUtilsFactory()).thenReturn(coordinationUtilsFactory);
+     PowerMockito.whenNew(JobCoordinatorConfig.class).withAnyArguments().thenReturn(mockJcConfig);
  
 -    LocalApplicationRunner spy = spy(runner);
      try {
 -      spy.run(app);
 +      runner.run(app);
      } catch (Throwable t) {
        assertNotNull(t); //no jobs exception
      }

http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
----------------------------------------------------------------------
diff --cc samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
index 8493cf1,0000000..d2f0184
mode 100644,000000..100644
--- a/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
+++ b/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
@@@ -1,103 -1,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.test.controlmessages;
 +
 +import java.util.ArrayList;
 +import java.util.HashMap;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Random;
 +import org.apache.samza.application.StreamApplication;
 +import org.apache.samza.config.JobConfig;
 +import org.apache.samza.config.JobCoordinatorConfig;
 +import org.apache.samza.config.MapConfig;
 +import org.apache.samza.config.TaskConfig;
 +import org.apache.samza.container.grouper.task.SingleContainerGrouperFactory;
++import org.apache.samza.operators.KV;
++import org.apache.samza.operators.functions.MapFunction;
 +import org.apache.samza.runtime.LocalApplicationRunner;
 +import org.apache.samza.standalone.PassthroughCoordinationUtilsFactory;
 +import org.apache.samza.standalone.PassthroughJobCoordinatorFactory;
 +import org.apache.samza.test.controlmessages.TestData.PageView;
 +import org.apache.samza.test.controlmessages.TestData.PageViewJsonSerdeFactory;
 +import org.apache.samza.test.harness.AbstractIntegrationTestHarness;
 +import org.apache.samza.test.util.ArraySystemFactory;
 +import org.apache.samza.test.util.Base64Serializer;
 +import org.junit.Test;
 +
 +import static org.junit.Assert.assertEquals;
 +
 +
 +/**
 + * This test uses an array as a bounded input source, and does a partitionBy() and sink() after reading the input.
 + * It verifies the pipeline will stop and the number of output messages should equal to the input.
 + */
 +public class EndOfStreamIntegrationTest extends AbstractIntegrationTestHarness {
 +
 +  private static final String[] PAGEKEYS = {"inbox", "home", "search", "pymk", "group", "job"};
 +
 +  @Test
 +  public void testPipeline() throws  Exception {
 +    Random random = new Random();
 +    int count = 10;
 +    PageView[] pageviews = new PageView[count];
 +    for (int i = 0; i < count; i++) {
 +      String pagekey = PAGEKEYS[random.nextInt(PAGEKEYS.length - 1)];
 +      int memberId = random.nextInt(10);
 +      pageviews[i] = new PageView(pagekey, memberId);
 +    }
 +
 +    int partitionCount = 4;
 +    Map<String, String> configs = new HashMap<>();
 +    configs.put("systems.test.samza.factory", ArraySystemFactory.class.getName());
 +    configs.put("streams.PageView.samza.system", "test");
 +    configs.put("streams.PageView.source", Base64Serializer.serialize(pageviews));
 +    configs.put("streams.PageView.partitionCount", String.valueOf(partitionCount));
 +
 +    configs.put(JobConfig.JOB_NAME(), "test-eos-job");
 +    configs.put(JobConfig.PROCESSOR_ID(), "1");
 +    configs.put(JobCoordinatorConfig.JOB_COORDINATION_UTILS_FACTORY, PassthroughCoordinationUtilsFactory.class.getName());
 +    configs.put(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, PassthroughJobCoordinatorFactory.class.getName());
 +    configs.put(TaskConfig.GROUPER_FACTORY(), SingleContainerGrouperFactory.class.getName());
 +
 +    configs.put("systems.kafka.samza.factory", "org.apache.samza.system.kafka.KafkaSystemFactory");
 +    configs.put("systems.kafka.producer.bootstrap.servers", bootstrapUrl());
 +    configs.put("systems.kafka.consumer.zookeeper.connect", zkConnect());
 +    configs.put("systems.kafka.samza.key.serde", "int");
 +    configs.put("systems.kafka.samza.msg.serde", "json");
 +    configs.put("systems.kafka.default.stream.replication.factor", "1");
 +    configs.put("job.default.system", "kafka");
 +
 +    configs.put("serializers.registry.int.class", "org.apache.samza.serializers.IntegerSerdeFactory");
 +    configs.put("serializers.registry.json.class", PageViewJsonSerdeFactory.class.getName());
 +
 +    final LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs));
 +    List<PageView> received = new ArrayList<>();
 +    final StreamApplication app = (streamGraph, cfg) -> {
-       streamGraph.getInputStream("PageView", (k, v) -> (PageView) v)
-         .partitionBy(PageView::getMemberId)
++      streamGraph.<KV<String, PageView>>getInputStream("PageView")
++        .map(Values.create())
++        .partitionBy(pv -> pv.getMemberId(), pv -> pv)
 +        .sink((m, collector, coordinator) -> {
-             received.add(m);
++            received.add(m.getValue());
 +          });
 +    };
 +    runner.run(app);
 +    runner.waitForFinish();
 +
 +    assertEquals(received.size(), count * partitionCount);
 +  }
++
++  public static final class Values {
++    public static <K, V, M extends KV<K, V>> MapFunction<M, V> create() {
++      return (M m) -> m.getValue();
++    }
++  }
 +}

http://git-wip-us.apache.org/repos/asf/samza/blob/052a0570/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java
----------------------------------------------------------------------
diff --cc samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java
index d9202d3,0000000..7da0e77
mode 100644,000000..100644
--- a/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java
+++ b/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java
@@@ -1,204 -1,0 +1,206 @@@
 +/*
 + * 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.test.controlmessages;
 +
 +import java.lang.reflect.Field;
 +import java.util.ArrayList;
 +import java.util.HashMap;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Set;
 +import org.apache.samza.Partition;
 +import org.apache.samza.application.StreamApplication;
 +import org.apache.samza.config.Config;
 +import org.apache.samza.config.JobConfig;
 +import org.apache.samza.config.JobCoordinatorConfig;
 +import org.apache.samza.config.MapConfig;
 +import org.apache.samza.config.TaskConfig;
 +import org.apache.samza.container.SamzaContainer;
 +import org.apache.samza.container.TaskInstance;
 +import org.apache.samza.container.TaskName;
 +import org.apache.samza.container.grouper.task.SingleContainerGrouperFactory;
 +import org.apache.samza.metrics.MetricsRegistry;
++import org.apache.samza.operators.KV;
 +import org.apache.samza.operators.impl.InputOperatorImpl;
 +import org.apache.samza.operators.impl.OperatorImpl;
 +import org.apache.samza.operators.impl.OperatorImplGraph;
 +import org.apache.samza.operators.impl.TestOperatorImpl;
 +import org.apache.samza.operators.spec.OperatorSpec;
 +import org.apache.samza.processor.StreamProcessor;
 +import org.apache.samza.processor.TestStreamProcessorUtil;
 +import org.apache.samza.runtime.LocalApplicationRunner;
 +import org.apache.samza.runtime.TestLocalApplicationRunner;
 +import org.apache.samza.serializers.IntegerSerdeFactory;
 +import org.apache.samza.serializers.StringSerdeFactory;
 +import org.apache.samza.standalone.PassthroughCoordinationUtilsFactory;
 +import org.apache.samza.standalone.PassthroughJobCoordinatorFactory;
 +import org.apache.samza.system.IncomingMessageEnvelope;
 +import org.apache.samza.system.SystemAdmin;
 +import org.apache.samza.system.SystemConsumer;
 +import org.apache.samza.system.SystemFactory;
 +import org.apache.samza.system.SystemProducer;
 +import org.apache.samza.system.SystemStreamPartition;
 +import org.apache.samza.task.AsyncStreamTaskAdapter;
 +import org.apache.samza.task.StreamOperatorTask;
 +import org.apache.samza.task.TestStreamOperatorTask;
 +import org.apache.samza.test.controlmessages.TestData.PageView;
 +import org.apache.samza.test.controlmessages.TestData.PageViewJsonSerdeFactory;
 +import org.apache.samza.test.harness.AbstractIntegrationTestHarness;
 +import org.apache.samza.test.util.SimpleSystemAdmin;
 +import org.apache.samza.test.util.TestStreamConsumer;
 +import org.junit.Test;
 +import scala.collection.JavaConverters;
 +
 +import static org.junit.Assert.assertEquals;
 +
 +
 +public class WatermarkIntegrationTest extends AbstractIntegrationTestHarness {
 +
 +  private static int offset = 1;
 +  private static final String TEST_SYSTEM = "test";
 +  private static final String TEST_STREAM = "PageView";
 +  private static final int PARTITION_COUNT = 2;
 +  private static final SystemStreamPartition SSP0 = new SystemStreamPartition(TEST_SYSTEM, TEST_STREAM, new Partition(0));
 +  private static final SystemStreamPartition SSP1 = new SystemStreamPartition(TEST_SYSTEM, TEST_STREAM, new Partition(1));
 +
 +  private final static List<IncomingMessageEnvelope> TEST_DATA = new ArrayList<>();
 +  static {
 +    TEST_DATA.add(createIncomingMessage(new PageView("inbox", 1), SSP0));
 +    TEST_DATA.add(createIncomingMessage(new PageView("home", 2), SSP1));
 +    TEST_DATA.add(IncomingMessageEnvelope.buildWatermarkEnvelope(SSP0, 1));
 +    TEST_DATA.add(IncomingMessageEnvelope.buildWatermarkEnvelope(SSP1, 2));
 +    TEST_DATA.add(IncomingMessageEnvelope.buildWatermarkEnvelope(SSP0, 4));
 +    TEST_DATA.add(IncomingMessageEnvelope.buildWatermarkEnvelope(SSP1, 3));
 +    TEST_DATA.add(createIncomingMessage(new PageView("search", 3), SSP0));
 +    TEST_DATA.add(createIncomingMessage(new PageView("pymk", 4), SSP1));
 +    TEST_DATA.add(IncomingMessageEnvelope.buildEndOfStreamEnvelope(SSP0));
 +    TEST_DATA.add(IncomingMessageEnvelope.buildEndOfStreamEnvelope(SSP1));
 +  }
 +
 +  public final static class TestSystemFactory implements SystemFactory {
 +    @Override
 +    public SystemConsumer getConsumer(String systemName, Config config, MetricsRegistry registry) {
 +      return new TestStreamConsumer(TEST_DATA);
 +    }
 +
 +    @Override
 +    public SystemProducer getProducer(String systemName, Config config, MetricsRegistry registry) {
 +      return null;
 +    }
 +
 +    @Override
 +    public SystemAdmin getAdmin(String systemName, Config config) {
 +      return new SimpleSystemAdmin(config);
 +    }
 +  }
 +
 +  private static IncomingMessageEnvelope createIncomingMessage(Object message, SystemStreamPartition ssp) {
 +    return new IncomingMessageEnvelope(ssp, String.valueOf(offset++), "", message);
 +  }
 +
 +  @Test
 +  public void testWatermark() throws Exception {
 +    Map<String, String> configs = new HashMap<>();
 +    configs.put("systems.test.samza.factory", TestSystemFactory.class.getName());
 +    configs.put("streams.PageView.samza.system", "test");
 +    configs.put("streams.PageView.partitionCount", String.valueOf(PARTITION_COUNT));
 +
 +    configs.put(JobConfig.JOB_NAME(), "test-watermark-job");
 +    configs.put(JobConfig.PROCESSOR_ID(), "1");
 +    configs.put(JobCoordinatorConfig.JOB_COORDINATION_UTILS_FACTORY, PassthroughCoordinationUtilsFactory.class.getName());
 +    configs.put(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, PassthroughJobCoordinatorFactory.class.getName());
 +    configs.put(TaskConfig.GROUPER_FACTORY(), SingleContainerGrouperFactory.class.getName());
 +
 +    configs.put("systems.kafka.samza.factory", "org.apache.samza.system.kafka.KafkaSystemFactory");
 +    configs.put("systems.kafka.producer.bootstrap.servers", bootstrapUrl());
 +    configs.put("systems.kafka.consumer.zookeeper.connect", zkConnect());
 +    configs.put("systems.kafka.samza.key.serde", "int");
 +    configs.put("systems.kafka.samza.msg.serde", "json");
 +    configs.put("systems.kafka.default.stream.replication.factor", "1");
 +    configs.put("job.default.system", "kafka");
 +
 +    configs.put("serializers.registry.int.class", IntegerSerdeFactory.class.getName());
 +    configs.put("serializers.registry.string.class", StringSerdeFactory.class.getName());
 +    configs.put("serializers.registry.json.class", PageViewJsonSerdeFactory.class.getName());
 +
 +    final LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs));
 +    List<PageView> received = new ArrayList<>();
 +    final StreamApplication app = (streamGraph, cfg) -> {
-       streamGraph.getInputStream("PageView", (k, v) -> (PageView) v)
-           .partitionBy(PageView::getMemberId)
++      streamGraph.<KV<String, PageView>>getInputStream("PageView")
++          .map(EndOfStreamIntegrationTest.Values.create())
++          .partitionBy(pv -> pv.getMemberId(), pv -> pv)
 +          .sink((m, collector, coordinator) -> {
-               received.add(m);
++              received.add(m.getValue());
 +            });
 +    };
 +    runner.run(app);
 +    Map<String, StreamOperatorTask> tasks = getTaskOperationGraphs(runner);
 +
 +    runner.waitForFinish();
 +
 +    StreamOperatorTask task0 = tasks.get("Partition 0");
 +    OperatorImplGraph graph = TestStreamOperatorTask.getOperatorImplGraph(task0);
 +    OperatorImpl pb = getOperator(graph, OperatorSpec.OpCode.PARTITION_BY);
 +    assertEquals(TestOperatorImpl.getInputWatermark(pb), 4);
 +    assertEquals(TestOperatorImpl.getOutputWatermark(pb), 4);
 +    OperatorImpl sink = getOperator(graph, OperatorSpec.OpCode.SINK);
 +    assertEquals(TestOperatorImpl.getInputWatermark(sink), 3);
 +    assertEquals(TestOperatorImpl.getOutputWatermark(sink), 3);
 +
 +    StreamOperatorTask task1 = tasks.get("Partition 1");
 +    graph = TestStreamOperatorTask.getOperatorImplGraph(task1);
 +    pb = getOperator(graph, OperatorSpec.OpCode.PARTITION_BY);
 +    assertEquals(TestOperatorImpl.getInputWatermark(pb), 3);
 +    assertEquals(TestOperatorImpl.getOutputWatermark(pb), 3);
 +    sink = getOperator(graph, OperatorSpec.OpCode.SINK);
 +    assertEquals(TestOperatorImpl.getInputWatermark(sink), 3);
 +    assertEquals(TestOperatorImpl.getOutputWatermark(sink), 3);
 +  }
 +
 +  Map<String, StreamOperatorTask> getTaskOperationGraphs(LocalApplicationRunner runner) throws Exception {
 +    StreamProcessor processor = TestLocalApplicationRunner.getProcessors(runner).iterator().next();
 +    SamzaContainer container = TestStreamProcessorUtil.getContainer(processor);
 +    Map<TaskName, TaskInstance> taskInstances = JavaConverters.mapAsJavaMapConverter(container.getTaskInstances()).asJava();
 +    Map<String, StreamOperatorTask> tasks = new HashMap<>();
 +    for (Map.Entry<TaskName, TaskInstance> entry : taskInstances.entrySet()) {
 +      AsyncStreamTaskAdapter adapter = (AsyncStreamTaskAdapter) entry.getValue().task();
 +      Field field = AsyncStreamTaskAdapter.class.getDeclaredField("wrappedTask");
 +      field.setAccessible(true);
 +      StreamOperatorTask task = (StreamOperatorTask) field.get(adapter);
 +      tasks.put(entry.getKey().getTaskName(), task);
 +    }
 +    return tasks;
 +  }
 +
 +  OperatorImpl getOperator(OperatorImplGraph graph, OperatorSpec.OpCode opCode) {
 +    for (InputOperatorImpl input : graph.getAllInputOperators()) {
 +      Set<OperatorImpl> nextOps = TestOperatorImpl.getNextOperators(input);
 +      while (!nextOps.isEmpty()) {
 +        OperatorImpl op = nextOps.iterator().next();
 +        if (TestOperatorImpl.getOpCode(op) == opCode) {
 +          return op;
 +        } else {
 +          nextOps = TestOperatorImpl.getNextOperators(op);
 +        }
 +      }
 +    }
 +    return null;
 +  }
 +}


[05/16] samza git commit: Merge branch 'master' into 0.14.0

Posted by xi...@apache.org.
Merge branch 'master' into 0.14.0

Conflicts:
	samza-core/src/main/java/org/apache/samza/processor/StreamProcessor.java


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

Branch: refs/heads/master
Commit: 23bfaa8d117511ecc1e8a71c0d2695def24158af
Parents: bb3007d bf4c761
Author: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Authored: Wed Aug 9 10:39:55 2017 -0700
Committer: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Committed: Wed Aug 9 10:39:55 2017 -0700

----------------------------------------------------------------------
 .gitignore                                      |   2 +-
 build.gradle                                    |  20 ++
 .../versioned/container/metrics-table.html      |   8 +-
 .../versioned/jobs/configuration-table.html     |  32 +-
 .../autoscaling/deployer/ConfigManager.java     |   2 +-
 samza-azure/README.md                           |  34 +++
 .../main/java/org/apache/samza/AzureClient.java |  70 +++++
 .../main/java/org/apache/samza/AzureConfig.java |  73 +++++
 .../java/org/apache/samza/AzureException.java   |  43 +++
 .../main/java/org/apache/samza/BlobUtils.java   | 280 +++++++++++++++++
 .../java/org/apache/samza/JobModelBundle.java   |  61 ++++
 .../java/org/apache/samza/LeaseBlobManager.java |  98 ++++++
 .../java/org/apache/samza/ProcessorEntity.java  |  58 ++++
 .../main/java/org/apache/samza/TableUtils.java  | 198 ++++++++++++
 .../clustermanager/ContainerProcessManager.java | 145 +++++----
 .../clustermanager/SamzaApplicationState.java   |   8 +
 .../org/apache/samza/config/TaskConfigJava.java |   4 +-
 .../grouper/stream/GroupByPartition.java        |   2 +-
 .../grouper/task/GroupByContainerIds.java       |  18 +-
 .../apache/samza/operators/StreamGraphImpl.java |   4 +-
 .../samza/operators/impl/OperatorImpl.java      |   4 +-
 .../samza/operators/impl/OperatorImplGraph.java |   5 +-
 .../apache/samza/processor/StreamProcessor.java | 103 ++++---
 .../org/apache/samza/task/TaskFactoryUtil.java  |   5 +-
 .../java/org/apache/samza/zk/ProcessorData.java |  18 +-
 .../samza/zk/ScheduleAfterDebounceTime.java     |   8 +
 .../samza/zk/ZkBarrierForVersionUpgrade.java    |  33 +-
 .../org/apache/samza/zk/ZkControllerImpl.java   |  42 ++-
 .../samza/zk/ZkCoordinationServiceFactory.java  |   6 +-
 .../apache/samza/zk/ZkCoordinationUtils.java    |  12 +-
 .../org/apache/samza/zk/ZkJobCoordinator.java   | 150 +++++++---
 .../samza/zk/ZkJobCoordinatorFactory.java       |  22 +-
 .../samza/zk/ZkJobCoordinatorMetrics.java       |   9 -
 .../org/apache/samza/zk/ZkLeaderElector.java    |  19 +-
 .../main/java/org/apache/samza/zk/ZkUtils.java  | 300 +++++++++++++++++--
 .../org/apache/samza/zk/ZkUtilsMetrics.java     |  56 ++++
 .../org/apache/samza/config/JobConfig.scala     |   2 +-
 .../apache/samza/container/SamzaContainer.scala |   4 +-
 .../ContainerProcessManagerMetrics.scala        |   1 +
 .../samza/serializers/ByteBufferSerde.scala     |   5 +-
 .../apache/samza/system/SystemConsumers.scala   |   2 +-
 .../clustermanager/MockContainerAllocator.java  |  23 ++
 .../TestContainerProcessManager.java            | 207 +++++++++++--
 .../grouper/task/TestGroupByContainerIds.java   |  31 +-
 .../samza/operators/TestJoinOperator.java       |   2 +-
 .../operators/impl/TestOperatorImplGraph.java   |  25 +-
 .../apache/samza/task/TestTaskFactoryUtil.java  |  11 +
 .../zk/TestZkBarrierForVersionUpgrade.java      |   4 +-
 .../apache/samza/zk/TestZkJobCoordinator.java   |  49 +++
 .../apache/samza/zk/TestZkLeaderElector.java    |   2 +-
 .../apache/samza/zk/TestZkProcessorLatch.java   |   4 +-
 .../java/org/apache/samza/zk/TestZkUtils.java   | 148 ++++++++-
 .../samza/serializers/TestByteBufferSerde.scala |  24 +-
 .../hdfs/reader/TestMultiFileHdfsReader.java    |   6 +-
 .../system/kafka/KafkaSystemConsumer.scala      |   2 +-
 .../storage/kv/KeyValueStorageEngine.scala      |   2 +-
 .../samza/rest/proxy/job/ScriptJobProxy.java    |   1 +
 .../apache/samza/rest/script/ScriptRunner.java  |  29 +-
 .../samza/processor/TestZkStreamProcessor.java  | 251 ++++++++++++++++
 .../processor/TestZkStreamProcessorBase.java    |  88 ++++--
 .../TestZkStreamProcessorFailures.java          | 151 ++++++++++
 .../processor/TestZkStreamProcessorSession.java | 135 +++++++++
 .../processor/TestZkLocalApplicationRunner.java | 242 ++++++++++-----
 .../test/processor/TestZkStreamProcessor.java   | 248 ---------------
 .../TestZkStreamProcessorFailures.java          | 147 ---------
 settings.gradle                                 |   3 +-
 66 files changed, 2975 insertions(+), 826 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/23bfaa8d/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/23bfaa8d/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/23bfaa8d/samza-core/src/main/java/org/apache/samza/processor/StreamProcessor.java
----------------------------------------------------------------------
diff --cc samza-core/src/main/java/org/apache/samza/processor/StreamProcessor.java
index 653c0bb,415111f..70be208
--- a/samza-core/src/main/java/org/apache/samza/processor/StreamProcessor.java
+++ b/samza-core/src/main/java/org/apache/samza/processor/StreamProcessor.java
@@@ -67,10 -69,10 +68,10 @@@ public class StreamProcessor 
  
    private volatile SamzaContainer container = null;
    private volatile Throwable containerException = null;
--  
++
    // Latch used to synchronize between the JobCoordinator thread and the container thread, when the container is
    // stopped due to re-balancing
-   private volatile CountDownLatch jcContainerShutdownLatch = new CountDownLatch(1);
+   /* package private */volatile CountDownLatch jcContainerShutdownLatch;
    private volatile boolean processorOnStartCalled = false;
  
    @VisibleForTesting
@@@ -233,62 -244,59 +243,57 @@@
  
        @Override
        public void onNewJobModel(String processorId, JobModel jobModel) {
-         if (!jobModel.getContainers().containsKey(processorId)) {
-           LOGGER.warn("JobModel does not contain the processorId: " + processorId + ". Stopping the processor.");
-           stop();
-         } else {
-           jcContainerShutdownLatch = new CountDownLatch(1);
- 
-           SamzaContainerListener containerListener = new SamzaContainerListener() {
-             @Override
-             public void onContainerStart() {
-               if (!processorOnStartCalled) {
-                 // processorListener is called on start only the first time the container starts.
-                 // It is not called after every re-balance of partitions among the processors
-                 processorOnStartCalled = true;
-                 if (processorListener != null) {
-                   processorListener.onStart();
-                 }
-               } else {
-                 LOGGER.debug("StreamProcessorListener was notified of container start previously. Hence, skipping this time.");
-               }
-             }
+         jcContainerShutdownLatch = new CountDownLatch(1);
  
-             @Override
-             public void onContainerStop(boolean pauseByJm) {
-               if (pauseByJm) {
-                 LOGGER.info("Container " + container.toString() + " stopped due to a request from JobCoordinator.");
-                 if (jcContainerShutdownLatch != null) {
-                   jcContainerShutdownLatch.countDown();
-                 }
-               } else {  // sp.stop was called or container stopped by itself
-                 LOGGER.info("Container " + container.toString() + " stopped.");
-                 container = null; // this guarantees that stop() doesn't try to stop container again
-                 stop();
+         SamzaContainerListener containerListener = new SamzaContainerListener() {
+           @Override
+           public void onContainerStart() {
+             if (!processorOnStartCalled) {
+               // processorListener is called on start only the first time the container starts.
+               // It is not called after every re-balance of partitions among the processors
+               processorOnStartCalled = true;
+               if (processorListener != null) {
+                 processorListener.onStart();
                }
+             } else {
+               LOGGER.debug("StreamProcessorListener was notified of container start previously. Hence, skipping this time.");
              }
+           }
  
-             @Override
-             public void onContainerFailed(Throwable t) {
+           @Override
+           public void onContainerStop(boolean pauseByJm) {
+             if (pauseByJm) {
+               LOGGER.info("Container " + container.toString() + " stopped due to a request from JobCoordinator.");
                if (jcContainerShutdownLatch != null) {
                  jcContainerShutdownLatch.countDown();
-               } else {
-                 LOGGER.warn("JobCoordinatorLatch was null. It is possible for some component to be waiting.");
                }
-               containerException = t;
-               LOGGER.error("Container failed. Stopping the processor.", containerException);
-               container = null;
+             } else {  // sp.stop was called or container stopped by itself
+               LOGGER.info("Container " + container.toString() + " stopped.");
+               container = null; // this guarantees that stop() doesn't try to stop container again
                stop();
              }
-           };
+           }
  
-           container = createSamzaContainer(processorId, jobModel);
-           container.setContainerListener(containerListener);
-           LOGGER.info("Starting container " + container.toString());
-           executorService = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
-               .setNameFormat("p-" + processorId + "-container-thread-%d").build());
-           executorService.submit(container::run);
-         }
+           @Override
+           public void onContainerFailed(Throwable t) {
+             if (jcContainerShutdownLatch != null) {
+               jcContainerShutdownLatch.countDown();
+             } else {
+               LOGGER.warn("JobCoordinatorLatch was null. It is possible for some component to be waiting.");
+             }
+             containerException = t;
+             LOGGER.error("Container failed. Stopping the processor.", containerException);
+             container = null;
+             stop();
+           }
+         };
+ 
 -        container = createSamzaContainer(
 -            jobModel.getContainers().get(processorId),
 -            jobModel.maxChangeLogStreamPartitions);
++        container = createSamzaContainer(processorId, jobModel);
+         container.setContainerListener(containerListener);
+         LOGGER.info("Starting container " + container.toString());
+         executorService = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+             .setNameFormat("p-" + processorId + "-container-thread-%d").build());
+         executorService.submit(container::run);
        }
  
        @Override

http://git-wip-us.apache.org/repos/asf/samza/blob/23bfaa8d/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala
----------------------------------------------------------------------


[03/16] samza git commit: SAMZA-1321: Propagate end-of-stream and watermark messages

Posted by xi...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/test/java/org/apache/samza/control/TestEndOfStreamManager.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/control/TestEndOfStreamManager.java b/samza-core/src/test/java/org/apache/samza/control/TestEndOfStreamManager.java
new file mode 100644
index 0000000..cc70b6b
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/control/TestEndOfStreamManager.java
@@ -0,0 +1,333 @@
+/*
+ * 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.control;
+
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.samza.Partition;
+import org.apache.samza.container.TaskName;
+import org.apache.samza.message.EndOfStreamMessage;
+import org.apache.samza.operators.spec.OperatorSpecs;
+import org.apache.samza.operators.spec.OutputOperatorSpec;
+import org.apache.samza.operators.spec.OutputStreamImpl;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.StreamMetadataCache;
+import org.apache.samza.system.StreamSpec;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamMetadata;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+
+public class TestEndOfStreamManager {
+  StreamMetadataCache metadataCache;
+
+  @Before
+  public void setup() {
+    SystemStreamMetadata metadata = mock(SystemStreamMetadata.class);
+    Map<Partition, SystemStreamMetadata.SystemStreamPartitionMetadata> partitionMetadata = new HashMap<>();
+    partitionMetadata.put(new Partition(0), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
+    partitionMetadata.put(new Partition(1), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
+    partitionMetadata.put(new Partition(2), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
+    partitionMetadata.put(new Partition(3), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
+    when(metadata.getSystemStreamPartitionMetadata()).thenReturn(partitionMetadata);
+    metadataCache = mock(StreamMetadataCache.class);
+    when(metadataCache.getSystemStreamMetadata(anyObject(), anyBoolean())).thenReturn(metadata);
+  }
+
+  @Test
+  public void testUpdateFromInputSource() {
+    SystemStreamPartition ssp = new SystemStreamPartition("test-system", "test-stream", new Partition(0));
+    TaskName taskName = new TaskName("Task 0");
+    Multimap<SystemStream, String> streamToTasks = HashMultimap.create();
+    streamToTasks.put(ssp.getSystemStream(), taskName.getTaskName());
+    ControlMessageListenerTask listener = mock(ControlMessageListenerTask.class);
+    when(listener.getIOGraph()).thenReturn(new IOGraph(Collections.emptyList()));
+    EndOfStreamManager manager = new EndOfStreamManager("Task 0", listener, streamToTasks, Collections.singleton(ssp), null, null);
+    manager.update(EndOfStreamManager.buildEndOfStreamEnvelope(ssp), mock(TaskCoordinator.class));
+    assertTrue(manager.isEndOfStream(ssp.getSystemStream()));
+  }
+
+  @Test
+  public void testUpdateFromIntermediateStream() {
+    SystemStreamPartition[] ssps = new SystemStreamPartition[3];
+    ssps[0] = new SystemStreamPartition("test-system", "test-stream-1", new Partition(0));
+    ssps[1] = new SystemStreamPartition("test-system", "test-stream-2", new Partition(0));
+    ssps[2] = new SystemStreamPartition("test-system", "test-stream-2", new Partition(1));
+
+    TaskName taskName = new TaskName("Task 0");
+    Multimap<SystemStream, String> streamToTasks = HashMultimap.create();
+    for (SystemStreamPartition ssp : ssps) {
+      streamToTasks.put(ssp.getSystemStream(), taskName.getTaskName());
+    }
+    ControlMessageListenerTask listener = mock(ControlMessageListenerTask.class);
+    when(listener.getIOGraph()).thenReturn(new IOGraph(Collections.emptyList()));
+    EndOfStreamManager manager = new EndOfStreamManager("Task 0", listener, streamToTasks, new HashSet<>(Arrays.asList(ssps)), null, null);
+
+    int envelopeCount = 4;
+    IncomingMessageEnvelope[] envelopes = new IncomingMessageEnvelope[envelopeCount];
+    for (int i = 0; i < envelopeCount; i++) {
+      envelopes[i] = new IncomingMessageEnvelope(ssps[0], "dummy-offset", "", new EndOfStreamMessage("task " + i, envelopeCount));
+    }
+    TaskCoordinator coordinator = mock(TaskCoordinator.class);
+
+    // verify the first three messages won't result in end-of-stream
+    for (int i = 0; i < 3; i++) {
+      manager.update(envelopes[i], coordinator);
+      assertFalse(manager.isEndOfStream(ssps[0].getSystemStream()));
+    }
+    // the fourth message will end the stream
+    manager.update(envelopes[3], coordinator);
+    assertTrue(manager.isEndOfStream(ssps[0].getSystemStream()));
+    assertFalse(manager.isEndOfStream(ssps[1].getSystemStream()));
+
+    // stream2 has two partitions assigned to this task, so it requires a message from each partition to end it
+    envelopes = new IncomingMessageEnvelope[envelopeCount];
+    for (int i = 0; i < envelopeCount; i++) {
+      envelopes[i] = new IncomingMessageEnvelope(ssps[1], "dummy-offset", "dummy-key", new EndOfStreamMessage("task " + i, envelopeCount));
+    }
+    // verify the messages for the partition 0 won't result in end-of-stream
+    for (int i = 0; i < 4; i++) {
+      manager.update(envelopes[i], coordinator);
+      assertFalse(manager.isEndOfStream(ssps[1].getSystemStream()));
+    }
+    for (int i = 0; i < envelopeCount; i++) {
+      envelopes[i] = new IncomingMessageEnvelope(ssps[2], "dummy-offset", "dummy-key", new EndOfStreamMessage("task " + i, envelopeCount));
+    }
+    for (int i = 0; i < 3; i++) {
+      manager.update(envelopes[i], coordinator);
+      assertFalse(manager.isEndOfStream(ssps[1].getSystemStream()));
+    }
+    // the fourth message will end the stream
+    manager.update(envelopes[3], coordinator);
+    assertTrue(manager.isEndOfStream(ssps[1].getSystemStream()));
+  }
+
+  @Test
+  public void testUpdateFromIntermediateStreamWith2Tasks() {
+    SystemStreamPartition[] ssps0 = new SystemStreamPartition[2];
+    ssps0[0] = new SystemStreamPartition("test-system", "test-stream-1", new Partition(0));
+    ssps0[1] = new SystemStreamPartition("test-system", "test-stream-2", new Partition(0));
+
+    SystemStreamPartition ssp1 = new SystemStreamPartition("test-system", "test-stream-2", new Partition(1));
+
+    TaskName t0 = new TaskName("Task 0");
+    Multimap<SystemStream, String> streamToTasks = HashMultimap.create();
+    for (SystemStreamPartition ssp : ssps0) {
+      streamToTasks.put(ssp.getSystemStream(), t0.getTaskName());
+    }
+
+    TaskName t1 = new TaskName("Task 1");
+    streamToTasks.put(ssp1, t1.getTaskName());
+
+    List<StreamSpec> inputs = new ArrayList<>();
+    inputs.add(new StreamSpec("test-stream-1", "test-stream-1", "test-system"));
+    inputs.add(new StreamSpec("test-stream-2", "test-stream-2", "test-system"));
+    StreamSpec outputSpec = new StreamSpec("int-stream", "int-stream", "test-system");
+    IOGraph ioGraph = TestIOGraph.buildSimpleIOGraph(inputs, outputSpec, true);
+
+    ControlMessageListenerTask listener = mock(ControlMessageListenerTask.class);
+    when(listener.getIOGraph()).thenReturn(ioGraph);
+
+    EndOfStreamManager manager0 = spy(new EndOfStreamManager("Task 0", listener, streamToTasks, new HashSet<>(Arrays.asList(ssps0)), null, null));
+    manager0.update(EndOfStreamManager.buildEndOfStreamEnvelope(ssps0[0]), mock(TaskCoordinator.class));
+    assertTrue(manager0.isEndOfStream(ssps0[0].getSystemStream()));
+    doNothing().when(manager0).sendEndOfStream(any(), anyInt());
+    manager0.update(EndOfStreamManager.buildEndOfStreamEnvelope(ssps0[1]), mock(TaskCoordinator.class));
+    assertTrue(manager0.isEndOfStream(ssps0[1].getSystemStream()));
+    verify(manager0).sendEndOfStream(any(), anyInt());
+
+    EndOfStreamManager manager1 = spy(new EndOfStreamManager("Task 1", listener, streamToTasks, Collections.singleton(
+        ssp1), null, null));
+    doNothing().when(manager1).sendEndOfStream(any(), anyInt());
+    manager1.update(EndOfStreamManager.buildEndOfStreamEnvelope(ssp1), mock(TaskCoordinator.class));
+    assertTrue(manager1.isEndOfStream(ssp1.getSystemStream()));
+    verify(manager1).sendEndOfStream(any(), anyInt());
+  }
+
+  @Test
+  public void testSendEndOfStream() {
+    StreamSpec ints = new StreamSpec("int-stream", "int-stream", "test-system");
+    StreamSpec input = new StreamSpec("input-stream", "input-stream", "test-system");
+    IOGraph ioGraph = TestIOGraph.buildSimpleIOGraph(Collections.singletonList(input), ints, true);
+
+    Multimap<SystemStream, String> inputToTasks = HashMultimap.create();
+    for (int i = 0; i < 8; i++) {
+      inputToTasks.put(input.toSystemStream(), "Task " + i);
+    }
+
+    MessageCollector collector = mock(MessageCollector.class);
+    TaskName taskName = new TaskName("Task 0");
+    ControlMessageListenerTask listener = mock(ControlMessageListenerTask.class);
+    when(listener.getIOGraph()).thenReturn(ioGraph);
+    EndOfStreamManager manager = new EndOfStreamManager(taskName.getTaskName(),
+        listener,
+        inputToTasks,
+        Collections.EMPTY_SET,
+        metadataCache,
+        collector);
+
+    Set<Integer> partitions = new HashSet<>();
+    doAnswer(invocation -> {
+        OutgoingMessageEnvelope envelope = (OutgoingMessageEnvelope) invocation.getArguments()[0];
+        partitions.add((Integer) envelope.getPartitionKey());
+        EndOfStreamMessage eosMessage = (EndOfStreamMessage) envelope.getMessage();
+        assertEquals(eosMessage.getTaskName(), taskName.getTaskName());
+        assertEquals(eosMessage.getTaskCount(), 8);
+        return null;
+      }).when(collector).send(any());
+
+    manager.sendEndOfStream(input.toSystemStream(), 8);
+    assertEquals(partitions.size(), 4);
+  }
+
+  @Test
+  public void testPropagate() {
+    List<StreamSpec> inputs = new ArrayList<>();
+    inputs.add(new StreamSpec("input-stream-1", "input-stream-1", "test-system"));
+    inputs.add(new StreamSpec("input-stream-2", "input-stream-2", "test-system"));
+    StreamSpec outputSpec = new StreamSpec("int-stream", "int-stream", "test-system");
+
+    SystemStream input1 = new SystemStream("test-system", "input-stream-1");
+    SystemStream input2 = new SystemStream("test-system", "input-stream-2");
+    SystemStream ints = new SystemStream("test-system", "int-stream");
+    SystemStreamPartition[] ssps = new SystemStreamPartition[3];
+    ssps[0] = new SystemStreamPartition(input1, new Partition(0));
+    ssps[1] = new SystemStreamPartition(input2, new Partition(0));
+    ssps[2] = new SystemStreamPartition(ints, new Partition(0));
+
+    Set<SystemStreamPartition> sspSet = new HashSet<>(Arrays.asList(ssps));
+    TaskName taskName = new TaskName("task 0");
+    Multimap<SystemStream, String> streamToTasks = HashMultimap.create();
+    for (SystemStreamPartition ssp : ssps) {
+      streamToTasks.put(ssp.getSystemStream(), taskName.getTaskName());
+    }
+
+    IOGraph ioGraph = TestIOGraph.buildSimpleIOGraph(inputs, outputSpec, true);
+    MessageCollector collector = mock(MessageCollector.class);
+    ControlMessageListenerTask listener = mock(ControlMessageListenerTask.class);
+    when(listener.getIOGraph()).thenReturn(ioGraph);
+    EndOfStreamManager manager = spy(
+        new EndOfStreamManager("task 0", listener, streamToTasks, sspSet, metadataCache, collector));
+    TaskCoordinator coordinator = mock(TaskCoordinator.class);
+
+    // ssp1 end-of-stream, wait for ssp2
+    manager.update(EndOfStreamManager.buildEndOfStreamEnvelope(ssps[0]), coordinator);
+    verify(manager, never()).sendEndOfStream(any(), anyInt());
+
+    // ssp2 end-of-stream, propagate to intermediate
+    manager.update(EndOfStreamManager.buildEndOfStreamEnvelope(ssps[1]), coordinator);
+    doNothing().when(manager).sendEndOfStream(any(), anyInt());
+    ArgumentCaptor<SystemStream> argument = ArgumentCaptor.forClass(SystemStream.class);
+    verify(manager).sendEndOfStream(argument.capture(), anyInt());
+    assertEquals(ints, argument.getValue());
+
+    // intermediate end-of-stream, shutdown the task
+    manager.update(EndOfStreamManager.buildEndOfStreamEnvelope(ssps[2]), coordinator);
+    doNothing().when(coordinator).shutdown(any());
+    ArgumentCaptor<TaskCoordinator.RequestScope> arg = ArgumentCaptor.forClass(TaskCoordinator.RequestScope.class);
+    verify(coordinator).shutdown(arg.capture());
+    assertEquals(TaskCoordinator.RequestScope.CURRENT_TASK, arg.getValue());
+  }
+
+  //  Test the case when the publishing tasks to intermediate stream is a subset of total tasks
+  @Test
+  public void testPropogateWith2Tasks() {
+    StreamSpec outputSpec = new StreamSpec("int-stream", "int-stream", "test-system");
+    OutputStreamImpl outputStream = new OutputStreamImpl(outputSpec, null, null);
+    OutputOperatorSpec partitionByOp = OperatorSpecs.createPartitionByOperatorSpec(outputStream, 0);
+
+    List<StreamSpec> inputs = new ArrayList<>();
+    inputs.add(new StreamSpec("input-stream-1", "input-stream-1", "test-system"));
+
+    IOGraph ioGraph = TestIOGraph.buildSimpleIOGraph(inputs, outputSpec, true);
+
+    SystemStream input1 = new SystemStream("test-system", "input-stream-1");
+    SystemStream ints = new SystemStream("test-system", "int-stream");
+    SystemStreamPartition ssp1 = new SystemStreamPartition(input1, new Partition(0));
+    SystemStreamPartition ssp2 = new SystemStreamPartition(ints, new Partition(0));
+
+    TaskName t0 = new TaskName("task 0");
+    TaskName t1 = new TaskName("task 1");
+    Multimap<SystemStream, String> streamToTasks = HashMultimap.create();
+    streamToTasks.put(ssp1.getSystemStream(), t0.getTaskName());
+    streamToTasks.put(ssp2.getSystemStream(), t1.getTaskName());
+
+    ControlMessageListenerTask listener = mock(ControlMessageListenerTask.class);
+    when(listener.getIOGraph()).thenReturn(ioGraph);
+
+    EndOfStreamManager manager0 = spy(
+        new EndOfStreamManager(t0.getTaskName(), listener, streamToTasks, Collections.singleton(ssp1), metadataCache, null));
+    EndOfStreamManager manager1 = spy(
+        new EndOfStreamManager(t1.getTaskName(), listener, streamToTasks, Collections.singleton(ssp2), metadataCache, null));
+
+    TaskCoordinator coordinator0 = mock(TaskCoordinator.class);
+    TaskCoordinator coordinator1 = mock(TaskCoordinator.class);
+
+    // ssp1 end-of-stream
+    doNothing().when(manager0).sendEndOfStream(any(), anyInt());
+    doNothing().when(coordinator0).shutdown(any());
+    manager0.update(EndOfStreamManager.buildEndOfStreamEnvelope(ssp1), coordinator0);
+    //verify task count is 1
+    ArgumentCaptor<Integer> argument = ArgumentCaptor.forClass(Integer.class);
+    verify(manager0).sendEndOfStream(any(), argument.capture());
+    assertTrue(argument.getValue() == 1);
+    ArgumentCaptor<TaskCoordinator.RequestScope> arg = ArgumentCaptor.forClass(TaskCoordinator.RequestScope.class);
+    verify(coordinator0).shutdown(arg.capture());
+    assertEquals(TaskCoordinator.RequestScope.CURRENT_TASK, arg.getValue());
+
+    // int1 end-of-stream
+    IncomingMessageEnvelope intEos = new IncomingMessageEnvelope(ssp2, null, null, new EndOfStreamMessage(t0.getTaskName(), 1));
+    manager1.update(intEos, coordinator1);
+    doNothing().when(coordinator1).shutdown(any());
+    verify(manager1, never()).sendEndOfStream(any(), anyInt());
+    arg = ArgumentCaptor.forClass(TaskCoordinator.RequestScope.class);
+    verify(coordinator1).shutdown(arg.capture());
+    assertEquals(TaskCoordinator.RequestScope.CURRENT_TASK, arg.getValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/test/java/org/apache/samza/control/TestIOGraph.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/control/TestIOGraph.java b/samza-core/src/test/java/org/apache/samza/control/TestIOGraph.java
new file mode 100644
index 0000000..39c56c3
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/control/TestIOGraph.java
@@ -0,0 +1,200 @@
+/*
+ * 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.control;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.control.IOGraph.IONode;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.OutputStream;
+import org.apache.samza.operators.StreamGraphImpl;
+import org.apache.samza.operators.functions.JoinFunction;
+import org.apache.samza.runtime.ApplicationRunner;
+import org.apache.samza.system.StreamSpec;
+import org.junit.Before;
+import org.junit.Test;
+
+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 TestIOGraph {
+  StreamSpec input1;
+  StreamSpec input2;
+  StreamSpec input3;
+  StreamSpec output1;
+  StreamSpec output2;
+  StreamSpec int1;
+  StreamSpec int2;
+
+  StreamGraphImpl streamGraph;
+
+  @Before
+  public void setup() {
+    ApplicationRunner runner = mock(ApplicationRunner.class);
+    Map<String, String> configMap = new HashMap<>();
+    configMap.put(JobConfig.JOB_NAME(), "test-app");
+    configMap.put(JobConfig.JOB_DEFAULT_SYSTEM(), "test-system");
+    Config config = new MapConfig(configMap);
+
+    /**
+     * the graph looks like the following. number of partitions in parentheses. quotes indicate expected value.
+     *
+     *                                    input1 -> map -> join -> output1
+     *                                                       |
+     *                      input2 -> partitionBy -> filter -|
+     *                                                       |
+     *           input3 -> filter -> partitionBy -> map -> join -> output2
+     *
+     */
+    input1 = new StreamSpec("input1", "input1", "system1");
+    input2 = new StreamSpec("input2", "input2", "system2");
+    input3 = new StreamSpec("input3", "input3", "system2");
+
+    output1 = new StreamSpec("output1", "output1", "system1");
+    output2 = new StreamSpec("output2", "output2", "system2");
+
+    runner = mock(ApplicationRunner.class);
+    when(runner.getStreamSpec("input1")).thenReturn(input1);
+    when(runner.getStreamSpec("input2")).thenReturn(input2);
+    when(runner.getStreamSpec("input3")).thenReturn(input3);
+    when(runner.getStreamSpec("output1")).thenReturn(output1);
+    when(runner.getStreamSpec("output2")).thenReturn(output2);
+
+    // intermediate streams used in tests
+    int1 = new StreamSpec("test-app-1-partition_by-3", "test-app-1-partition_by-3", "default-system");
+    int2 = new StreamSpec("test-app-1-partition_by-8", "test-app-1-partition_by-8", "default-system");
+    when(runner.getStreamSpec("test-app-1-partition_by-3"))
+        .thenReturn(int1);
+    when(runner.getStreamSpec("test-app-1-partition_by-8"))
+        .thenReturn(int2);
+
+    streamGraph = new StreamGraphImpl(runner, config);
+    BiFunction msgBuilder = mock(BiFunction.class);
+    MessageStream m1 = streamGraph.getInputStream("input1", msgBuilder).map(m -> m);
+    MessageStream m2 = streamGraph.getInputStream("input2", msgBuilder).partitionBy(m -> "haha").filter(m -> true);
+    MessageStream m3 = streamGraph.getInputStream("input3", msgBuilder).filter(m -> true).partitionBy(m -> "hehe").map(m -> m);
+    Function mockFn = mock(Function.class);
+    OutputStream<Object, Object, Object> om1 = streamGraph.getOutputStream("output1", mockFn, mockFn);
+    OutputStream<Object, Object, Object> om2 = streamGraph.getOutputStream("output2", mockFn, mockFn);
+
+    m1.join(m2, mock(JoinFunction.class), Duration.ofHours(2)).sendTo(om1);
+    m3.join(m2, mock(JoinFunction.class), Duration.ofHours(1)).sendTo(om2);
+  }
+
+  @Test
+  public void testBuildIOGraph() {
+    IOGraph ioGraph = streamGraph.toIOGraph();
+    assertEquals(ioGraph.getNodes().size(), 4);
+
+    for (IONode node : ioGraph.getNodes()) {
+      if (node.getOutput().equals(output1)) {
+        assertEquals(node.getInputs().size(), 2);
+        assertFalse(node.isOutputIntermediate());
+        StreamSpec[] inputs = sort(node.getInputs());
+        assertEquals(inputs[0], input1);
+        assertEquals(inputs[1], int1);
+      } else if (node.getOutput().equals(output2)) {
+        assertEquals(node.getInputs().size(), 2);
+        assertFalse(node.isOutputIntermediate());
+        StreamSpec[] inputs = sort(node.getInputs());
+        assertEquals(inputs[0], int1);
+        assertEquals(inputs[1], int2);
+      } else if (node.getOutput().equals(int1)) {
+        assertEquals(node.getInputs().size(), 1);
+        assertTrue(node.isOutputIntermediate());
+        StreamSpec[] inputs = sort(node.getInputs());
+        assertEquals(inputs[0], input2);
+      } else if (node.getOutput().equals(int2)) {
+        assertEquals(node.getInputs().size(), 1);
+        assertTrue(node.isOutputIntermediate());
+        StreamSpec[] inputs = sort(node.getInputs());
+        assertEquals(inputs[0], input3);
+      }
+    }
+  }
+
+  @Test
+  public void testNodesOfInput() {
+    IOGraph ioGraph = streamGraph.toIOGraph();
+    Collection<IONode> nodes = ioGraph.getNodesOfInput(input1.toSystemStream());
+    assertEquals(nodes.size(), 1);
+    IONode node = nodes.iterator().next();
+    assertEquals(node.getOutput(), output1);
+    assertEquals(node.getInputs().size(), 2);
+    assertFalse(node.isOutputIntermediate());
+
+    nodes = ioGraph.getNodesOfInput(input2.toSystemStream());
+    assertEquals(nodes.size(), 1);
+    node = nodes.iterator().next();
+    assertEquals(node.getOutput(), int1);
+    assertEquals(node.getInputs().size(), 1);
+    assertTrue(node.isOutputIntermediate());
+
+    nodes = ioGraph.getNodesOfInput(int1.toSystemStream());
+    assertEquals(nodes.size(), 2);
+    nodes.forEach(n -> {
+        assertEquals(n.getInputs().size(), 2);
+      });
+
+    nodes = ioGraph.getNodesOfInput(input3.toSystemStream());
+    assertEquals(nodes.size(), 1);
+    node = nodes.iterator().next();
+    assertEquals(node.getOutput(), int2);
+    assertEquals(node.getInputs().size(), 1);
+    assertTrue(node.isOutputIntermediate());
+
+    nodes = ioGraph.getNodesOfInput(int2.toSystemStream());
+    assertEquals(nodes.size(), 1);
+    node = nodes.iterator().next();
+    assertEquals(node.getOutput(), output2);
+    assertEquals(node.getInputs().size(), 2);
+    assertFalse(node.isOutputIntermediate());
+  }
+
+  private static StreamSpec[] sort(Set<StreamSpec> specs) {
+    StreamSpec[] array = new StreamSpec[specs.size()];
+    specs.toArray(array);
+    Arrays.sort(array, (s1, s2) -> s1.getId().compareTo(s2.getId()));
+    return array;
+  }
+
+  public static IOGraph buildSimpleIOGraph(List<StreamSpec> inputs,
+      StreamSpec output,
+      boolean isOutputIntermediate) {
+    IONode node = new IONode(output, isOutputIntermediate);
+    inputs.forEach(input -> node.addInput(input));
+    return new IOGraph(Collections.singleton(node));
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/test/java/org/apache/samza/control/TestWatermarkManager.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/control/TestWatermarkManager.java b/samza-core/src/test/java/org/apache/samza/control/TestWatermarkManager.java
new file mode 100644
index 0000000..8fe7a16
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/control/TestWatermarkManager.java
@@ -0,0 +1,260 @@
+/*
+ * 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.control;
+
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.samza.Partition;
+import org.apache.samza.container.TaskName;
+import org.apache.samza.message.WatermarkMessage;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.StreamMetadataCache;
+import org.apache.samza.system.StreamSpec;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamMetadata;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.task.MessageCollector;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+
+public class TestWatermarkManager {
+
+  StreamMetadataCache metadataCache;
+
+  @Before
+  public void setup() {
+    SystemStreamMetadata metadata = mock(SystemStreamMetadata.class);
+    Map<Partition, SystemStreamMetadata.SystemStreamPartitionMetadata> partitionMetadata = new HashMap<>();
+    partitionMetadata.put(new Partition(0), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
+    partitionMetadata.put(new Partition(1), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
+    partitionMetadata.put(new Partition(2), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
+    partitionMetadata.put(new Partition(3), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
+    when(metadata.getSystemStreamPartitionMetadata()).thenReturn(partitionMetadata);
+    metadataCache = mock(StreamMetadataCache.class);
+    when(metadataCache.getSystemStreamMetadata(anyObject(), anyBoolean())).thenReturn(metadata);
+  }
+
+  @Test
+  public void testUpdateFromInputSource() {
+    SystemStreamPartition ssp = new SystemStreamPartition("test-system", "test-stream", new Partition(0));
+    TaskName taskName = new TaskName("Task 0");
+    Multimap<SystemStream, String> streamToTasks = HashMultimap.create();
+    streamToTasks.put(ssp.getSystemStream(), taskName.getTaskName());
+    ControlMessageListenerTask listener = mock(ControlMessageListenerTask.class);
+    when(listener.getIOGraph()).thenReturn(new IOGraph(Collections.emptyList()));
+    WatermarkManager manager = new WatermarkManager("Task 0", listener, streamToTasks, Collections.singleton(ssp), null, null);
+    long time = System.currentTimeMillis();
+    Watermark watermark = manager.update(WatermarkManager.buildWatermarkEnvelope(time, ssp));
+    assertEquals(watermark.getTimestamp(), time);
+  }
+
+  @Test
+  public void testUpdateFromIntermediateStream() {
+    SystemStreamPartition[] ssps = new SystemStreamPartition[3];
+    ssps[0] = new SystemStreamPartition("test-system", "test-stream-1", new Partition(0));
+    ssps[1] = new SystemStreamPartition("test-system", "test-stream-2", new Partition(0));
+    ssps[2] = new SystemStreamPartition("test-system", "test-stream-2", new Partition(1));
+
+    TaskName taskName = new TaskName("Task 0");
+    Multimap<SystemStream, String> streamToTasks = HashMultimap.create();
+    for (SystemStreamPartition ssp : ssps) {
+      streamToTasks.put(ssp.getSystemStream(), taskName.getTaskName());
+    }
+    ControlMessageListenerTask listener = mock(ControlMessageListenerTask.class);
+    when(listener.getIOGraph()).thenReturn(new IOGraph(Collections.emptyList()));
+
+    WatermarkManager manager = new WatermarkManager("Task 0", listener, streamToTasks, new HashSet<>(Arrays.asList(ssps)), null, null);
+    int envelopeCount = 4;
+    IncomingMessageEnvelope[] envelopes = new IncomingMessageEnvelope[envelopeCount];
+
+    long[] time = {300L, 200L, 100L, 400L};
+    for (int i = 0; i < envelopeCount; i++) {
+      envelopes[i] = new IncomingMessageEnvelope(ssps[0], "dummy-offset", "", new WatermarkMessage(time[i], "task " + i, envelopeCount));
+    }
+    for (int i = 0; i < 3; i++) {
+      assertNull(manager.update(envelopes[i]));
+    }
+    // verify the first three messages won't result in end-of-stream
+    assertEquals(manager.getWatermarkTime(ssps[0]), WatermarkManager.TIME_NOT_EXIST);
+    // the fourth message will generate a watermark
+    Watermark watermark = manager.update(envelopes[3]);
+    assertNotNull(watermark);
+    assertEquals(watermark.getTimestamp(), 100);
+    assertEquals(manager.getWatermarkTime(ssps[1]), WatermarkManager.TIME_NOT_EXIST);
+    assertEquals(manager.getWatermarkTime(ssps[2]), WatermarkManager.TIME_NOT_EXIST);
+
+
+    // stream2 has two partitions assigned to this task, so it requires a message from each partition to calculate watermarks
+    long[] time1 = {300L, 200L, 100L, 400L};
+    envelopes = new IncomingMessageEnvelope[envelopeCount];
+    for (int i = 0; i < envelopeCount; i++) {
+      envelopes[i] = new IncomingMessageEnvelope(ssps[1], "dummy-offset", "", new WatermarkMessage(time1[i], "task " + i, envelopeCount));
+    }
+    // verify the messages for the partition 0 won't generate watermark
+    for (int i = 0; i < 4; i++) {
+      assertNull(manager.update(envelopes[i]));
+    }
+    assertEquals(manager.getWatermarkTime(ssps[1]), 100L);
+
+    long[] time2 = {350L, 150L, 500L, 80L};
+    for (int i = 0; i < envelopeCount; i++) {
+      envelopes[i] = new IncomingMessageEnvelope(ssps[2], "dummy-offset", "", new WatermarkMessage(time2[i], "task " + i, envelopeCount));
+    }
+    for (int i = 0; i < 3; i++) {
+      assertNull(manager.update(envelopes[i]));
+    }
+    assertEquals(manager.getWatermarkTime(ssps[2]), WatermarkManager.TIME_NOT_EXIST);
+    // the fourth message will generate the watermark
+    watermark = manager.update(envelopes[3]);
+    assertNotNull(watermark);
+    assertEquals(manager.getWatermarkTime(ssps[2]), 80L);
+    assertEquals(watermark.getTimestamp(), 80L);
+  }
+
+  @Test
+  public void testSendWatermark() {
+    SystemStream ints = new SystemStream("test-system", "int-stream");
+    SystemStreamMetadata metadata = mock(SystemStreamMetadata.class);
+    Map<Partition, SystemStreamMetadata.SystemStreamPartitionMetadata> partitionMetadata = new HashMap<>();
+    partitionMetadata.put(new Partition(0), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
+    partitionMetadata.put(new Partition(1), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
+    partitionMetadata.put(new Partition(2), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
+    partitionMetadata.put(new Partition(3), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
+    when(metadata.getSystemStreamPartitionMetadata()).thenReturn(partitionMetadata);
+    StreamMetadataCache metadataCache = mock(StreamMetadataCache.class);
+    when(metadataCache.getSystemStreamMetadata(anyObject(), anyBoolean())).thenReturn(metadata);
+
+    MessageCollector collector = mock(MessageCollector.class);
+    ControlMessageListenerTask listener = mock(ControlMessageListenerTask.class);
+    when(listener.getIOGraph()).thenReturn(new IOGraph(Collections.emptyList()));
+
+    WatermarkManager manager = new WatermarkManager("task 0",
+        listener,
+        HashMultimap.create(),
+        Collections.EMPTY_SET,
+        metadataCache,
+        collector);
+
+    long time = System.currentTimeMillis();
+    Set<Integer> partitions = new HashSet<>();
+    doAnswer(invocation -> {
+        OutgoingMessageEnvelope envelope = (OutgoingMessageEnvelope) invocation.getArguments()[0];
+        partitions.add((Integer) envelope.getPartitionKey());
+        WatermarkMessage watermarkMessage = (WatermarkMessage) envelope.getMessage();
+        assertEquals(watermarkMessage.getTaskName(), "task 0");
+        assertEquals(watermarkMessage.getTaskCount(), 8);
+        assertEquals(watermarkMessage.getTimestamp(), time);
+        return null;
+      }).when(collector).send(any());
+
+    manager.sendWatermark(time, ints, 8);
+    assertEquals(partitions.size(), 4);
+  }
+
+  @Test
+  public void testPropagate() {
+    StreamSpec outputSpec = new StreamSpec("int-stream", "int-stream", "test-system");
+    List<StreamSpec> inputs = new ArrayList<>();
+    inputs.add(new StreamSpec("input-stream-1", "input-stream-1", "test-system"));
+    inputs.add(new StreamSpec("input-stream-2", "input-stream-2", "test-system"));
+
+    IOGraph ioGraph = TestIOGraph.buildSimpleIOGraph(inputs, outputSpec, true);
+
+    SystemStream input1 = new SystemStream("test-system", "input-stream-1");
+    SystemStream input2 = new SystemStream("test-system", "input-stream-2");
+    SystemStream input3 = new SystemStream("test-system", "input-stream-3");
+    SystemStream ints = new SystemStream("test-system", "int-stream");
+    SystemStreamPartition[] ssps0 = new SystemStreamPartition[3];
+    ssps0[0] = new SystemStreamPartition(input1, new Partition(0));
+    ssps0[1] = new SystemStreamPartition(input2, new Partition(0));
+    ssps0[2] = new SystemStreamPartition(ints, new Partition(0));
+
+    SystemStreamPartition[] ssps1 = new SystemStreamPartition[4];
+    ssps1[0] = new SystemStreamPartition(input1, new Partition(1));
+    ssps1[1] = new SystemStreamPartition(input2, new Partition(1));
+    ssps1[2] = new SystemStreamPartition(input3, new Partition(1));
+    ssps1[3] = new SystemStreamPartition(ints, new Partition(1));
+
+    SystemStreamPartition[] ssps2 = new SystemStreamPartition[2];
+    ssps2[0] = new SystemStreamPartition(input3, new Partition(2));
+    ssps2[1] = new SystemStreamPartition(ints, new Partition(2));
+
+
+    TaskName t0 = new TaskName("task 0"); //consume input1 and input2
+    TaskName t1 = new TaskName("task 1"); //consume input 1 and input2 and input 3
+    TaskName t2 = new TaskName("task 2"); //consume input2 and input 3
+    Multimap<SystemStream, String> inputToTasks = HashMultimap.create();
+    for (SystemStreamPartition ssp : ssps0) {
+      inputToTasks.put(ssp.getSystemStream(), t0.getTaskName());
+    }
+    for (SystemStreamPartition ssp : ssps1) {
+      inputToTasks.put(ssp.getSystemStream(), t1.getTaskName());
+    }
+    for (SystemStreamPartition ssp : ssps2) {
+      inputToTasks.put(ssp.getSystemStream(), t2.getTaskName());
+    }
+
+    ControlMessageListenerTask listener = mock(ControlMessageListenerTask.class);
+    when(listener.getIOGraph()).thenReturn(ioGraph);
+    WatermarkManager manager = spy(
+        new WatermarkManager(t0.getTaskName(), listener, inputToTasks, new HashSet<>(Arrays.asList(ssps0)), null, null));
+
+    IncomingMessageEnvelope envelope = WatermarkManager.buildWatermarkEnvelope(System.currentTimeMillis(), ssps0[0]);
+    doNothing().when(manager).sendWatermark(anyLong(), any(), anyInt());
+    Watermark watermark = manager.update(envelope);
+    assertNotNull(watermark);
+    long time = System.currentTimeMillis();
+    Watermark updatedWatermark = watermark.copyWithTimestamp(time);
+    updatedWatermark.propagate(ints);
+    ArgumentCaptor<Long> arg1 = ArgumentCaptor.forClass(Long.class);
+    ArgumentCaptor<SystemStream> arg2 = ArgumentCaptor.forClass(SystemStream.class);
+    ArgumentCaptor<Integer> arg3 = ArgumentCaptor.forClass(Integer.class);
+    verify(manager).sendWatermark(arg1.capture(), arg2.capture(), arg3.capture());
+    assertEquals(arg1.getValue().longValue(), time);
+    assertEquals(arg2.getValue(), ints);
+    assertEquals(arg3.getValue().intValue(), 2);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
index d50d271..3ae8f5b 100644
--- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
+++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
@@ -18,6 +18,7 @@
  */
 package org.apache.samza.operators.impl;
 
+import java.util.Set;
 import org.apache.samza.config.Config;
 import org.apache.samza.metrics.Counter;
 import org.apache.samza.metrics.MetricsRegistry;
@@ -210,5 +211,22 @@ public class TestOperatorImpl {
      super(OpCode.INPUT, 1);
     }
   }
+
+  public static Set<OperatorImpl> getNextOperators(OperatorImpl op) {
+    return op.registeredOperators;
+  }
+
+  public static OperatorSpec.OpCode getOpCode(OperatorImpl op) {
+    return op.getOperatorSpec().getOpCode();
+  }
+
+  public static long getInputWatermark(OperatorImpl op) {
+    return op.getInputWatermarkTime();
+  }
+
+  public static long getOutputWatermark(OperatorImpl op) {
+    return op.getOutputWatermarkTime();
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java b/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java
index 6a8d765..fc1259c 100644
--- a/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java
+++ b/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java
@@ -88,9 +88,7 @@ public class TestStreamProcessor {
     }
 
     @Override
-    SamzaContainer createSamzaContainer(
-        ContainerModel containerModel,
-        int maxChangelogStreamPartitions) {
+    SamzaContainer createSamzaContainer(String processorId, JobModel jobModel) {
       if (container == null) {
         RunLoop mockRunLoop = mock(RunLoop.class);
         doAnswer(invocation ->

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
index a04bd3b..4be4e73 100644
--- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
+++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
@@ -19,6 +19,7 @@
 
 package org.apache.samza.runtime;
 
+import java.util.Set;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.ApplicationConfig;
 import org.apache.samza.config.JobConfig;
@@ -330,7 +331,6 @@ public class TestLocalApplicationRunner {
         return null;
       }).when(sp).start();
 
-
     LocalApplicationRunner spy = spy(runner);
     doReturn(sp).when(spy).createStreamProcessor(anyObject(), anyObject(), captor.capture());
 
@@ -343,4 +343,8 @@ public class TestLocalApplicationRunner {
     assertEquals(spy.status(app), ApplicationStatus.UnsuccessfulFinish);
   }
 
+  public static Set<StreamProcessor> getProcessors(LocalApplicationRunner runner) {
+    return runner.getProcessors();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestIntermediateMessageSerde.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestIntermediateMessageSerde.java b/samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestIntermediateMessageSerde.java
index 5b76bba..7192525 100644
--- a/samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestIntermediateMessageSerde.java
+++ b/samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestIntermediateMessageSerde.java
@@ -26,7 +26,7 @@ import java.io.ObjectOutputStream;
 import java.io.Serializable;
 import org.apache.samza.message.EndOfStreamMessage;
 import org.apache.samza.message.WatermarkMessage;
-import org.apache.samza.message.MessageType;
+import org.apache.samza.message.IntermediateMessageType;
 import org.apache.samza.serializers.IntermediateMessageSerde;
 import org.apache.samza.serializers.Serde;
 import org.junit.Test;
@@ -96,7 +96,7 @@ public class TestIntermediateMessageSerde {
     TestUserMessage userMessage = new TestUserMessage(msg, 0, System.currentTimeMillis());
     byte[] bytes = imserde.toBytes(userMessage);
     TestUserMessage de = (TestUserMessage) imserde.fromBytes(bytes);
-    assertEquals(MessageType.of(de), MessageType.USER_MESSAGE);
+    assertEquals(IntermediateMessageType.of(de), IntermediateMessageType.USER_MESSAGE);
     assertEquals(de.getMessage(), msg);
     assertEquals(de.getOffset(), 0);
     assertTrue(de.getTimestamp() > 0);
@@ -109,7 +109,7 @@ public class TestIntermediateMessageSerde {
     WatermarkMessage watermark = new WatermarkMessage(System.currentTimeMillis(), taskName, 8);
     byte[] bytes = imserde.toBytes(watermark);
     WatermarkMessage de = (WatermarkMessage) imserde.fromBytes(bytes);
-    assertEquals(MessageType.of(de), MessageType.WATERMARK);
+    assertEquals(IntermediateMessageType.of(de), IntermediateMessageType.WATERMARK_MESSAGE);
     assertEquals(de.getTaskName(), taskName);
     assertEquals(de.getTaskCount(), 8);
     assertTrue(de.getTimestamp() > 0);
@@ -123,7 +123,7 @@ public class TestIntermediateMessageSerde {
     EndOfStreamMessage eos = new EndOfStreamMessage(taskName, 8);
     byte[] bytes = imserde.toBytes(eos);
     EndOfStreamMessage de = (EndOfStreamMessage) imserde.fromBytes(bytes);
-    assertEquals(MessageType.of(de), MessageType.END_OF_STREAM);
+    assertEquals(IntermediateMessageType.of(de), IntermediateMessageType.END_OF_STREAM_MESSAGE);
     assertEquals(de.getTaskName(), taskName);
     assertEquals(de.getTaskCount(), 8);
     assertEquals(de.getVersion(), 1);

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java b/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java
index 1afc26a..03931f1 100644
--- a/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java
+++ b/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java
@@ -39,6 +39,7 @@ import org.apache.samza.container.TaskInstance;
 import org.apache.samza.container.TaskInstanceExceptionHandler;
 import org.apache.samza.container.TaskInstanceMetrics;
 import org.apache.samza.container.TaskName;
+import org.apache.samza.control.EndOfStreamManager;
 import org.apache.samza.metrics.MetricsRegistryMap;
 import org.apache.samza.system.IncomingMessageEnvelope;
 import org.apache.samza.system.SystemConsumer;
@@ -78,15 +79,15 @@ public class TestAsyncRunLoop {
   private final IncomingMessageEnvelope envelope0 = new IncomingMessageEnvelope(ssp0, "0", "key0", "value0");
   private final IncomingMessageEnvelope envelope1 = new IncomingMessageEnvelope(ssp1, "1", "key1", "value1");
   private final IncomingMessageEnvelope envelope3 = new IncomingMessageEnvelope(ssp0, "1", "key0", "value0");
-  private final IncomingMessageEnvelope ssp0EndOfStream = IncomingMessageEnvelope.buildEndOfStreamEnvelope(ssp0);
-  private final IncomingMessageEnvelope ssp1EndOfStream = IncomingMessageEnvelope.buildEndOfStreamEnvelope(ssp1);
+  private final IncomingMessageEnvelope ssp0EndOfStream = EndOfStreamManager.buildEndOfStreamEnvelope(ssp0);
+  private final IncomingMessageEnvelope ssp1EndOfStream = EndOfStreamManager.buildEndOfStreamEnvelope(ssp1);
 
   TaskInstance createTaskInstance(AsyncStreamTask task, TaskName taskName, SystemStreamPartition ssp, OffsetManager manager, SystemConsumers consumers) {
     TaskInstanceMetrics taskInstanceMetrics = new TaskInstanceMetrics("task", new MetricsRegistryMap());
     scala.collection.immutable.Set<SystemStreamPartition> sspSet = JavaConverters.asScalaSetConverter(Collections.singleton(ssp)).asScala().toSet();
     return new TaskInstance(task, taskName, mock(Config.class), taskInstanceMetrics,
         null, consumers, mock(TaskInstanceCollector.class), mock(SamzaContainerContext.class),
-        manager, null, null, sspSet, new TaskInstanceExceptionHandler(taskInstanceMetrics, new scala.collection.immutable.HashSet<String>()));
+        manager, null, null, sspSet, new TaskInstanceExceptionHandler(taskInstanceMetrics, new scala.collection.immutable.HashSet<String>()), null, null);
   }
 
   TaskInstance createTaskInstance(AsyncStreamTask task, TaskName taskName, SystemStreamPartition ssp) {
@@ -569,7 +570,7 @@ public class TestAsyncRunLoop {
     SystemStreamPartition ssp2 = new SystemStreamPartition("system1", "stream2", p2);
     IncomingMessageEnvelope envelope1 = new IncomingMessageEnvelope(ssp2, "1", "key1", "message1");
     IncomingMessageEnvelope envelope2 = new IncomingMessageEnvelope(ssp2, "2", "key1", "message1");
-    IncomingMessageEnvelope envelope3 = IncomingMessageEnvelope.buildEndOfStreamEnvelope(ssp2);
+    IncomingMessageEnvelope envelope3 = EndOfStreamManager.buildEndOfStreamEnvelope(ssp2);
 
     Map<SystemStreamPartition, List<IncomingMessageEnvelope>> sspMap = new HashMap<>();
     List<IncomingMessageEnvelope> messageList = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/test/java/org/apache/samza/task/TestStreamOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/task/TestStreamOperatorTask.java b/samza-core/src/test/java/org/apache/samza/task/TestStreamOperatorTask.java
new file mode 100644
index 0000000..45b08d7
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/task/TestStreamOperatorTask.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.task;
+
+import org.apache.samza.operators.impl.OperatorImplGraph;
+
+
+public class TestStreamOperatorTask {
+
+  public static OperatorImplGraph getOperatorImplGraph(StreamOperatorTask task) {
+    return task.getOperatorImplGraph();
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala b/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
index 40974a6..9025077 100644
--- a/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
+++ b/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
@@ -19,12 +19,21 @@
 
 package org.apache.samza.container
 
+
+import java.util
+import java.util
+import java.util.Collections
 import java.util.concurrent.ConcurrentHashMap
+import com.google.common.collect.Multimap
 import org.apache.samza.SamzaException
 import org.apache.samza.Partition
 import org.apache.samza.checkpoint.OffsetManager
 import org.apache.samza.config.Config
 import org.apache.samza.config.MapConfig
+import org.apache.samza.control.ControlMessageUtils
+import org.apache.samza.job.model.ContainerModel
+import org.apache.samza.job.model.JobModel
+import org.apache.samza.job.model.TaskModel
 import org.apache.samza.metrics.Counter
 import org.apache.samza.metrics.Metric
 import org.apache.samza.metrics.MetricsRegistryMap
@@ -354,6 +363,36 @@ class TestTaskInstance {
     val expected = List(envelope1, envelope2, envelope4)
     assertEquals(expected, result.toList)
   }
+
+  @Test
+  def testBuildInputToTasks = {
+    val system: String = "test-system"
+    val stream0: String = "test-stream-0"
+    val stream1: String = "test-stream-1"
+
+    val ssp0: SystemStreamPartition = new SystemStreamPartition(system, stream0, new Partition(0))
+    val ssp1: SystemStreamPartition = new SystemStreamPartition(system, stream0, new Partition(1))
+    val ssp2: SystemStreamPartition = new SystemStreamPartition(system, stream1, new Partition(0))
+
+    val task0: TaskName = new TaskName("Task 0")
+    val task1: TaskName = new TaskName("Task 1")
+    val ssps: util.Set[SystemStreamPartition] = new util.HashSet[SystemStreamPartition]
+    ssps.add(ssp0)
+    ssps.add(ssp2)
+    val tm0: TaskModel = new TaskModel(task0, ssps, new Partition(0))
+    val cm0: ContainerModel = new ContainerModel("c0", 0, Collections.singletonMap(task0, tm0))
+    val tm1: TaskModel = new TaskModel(task1, Collections.singleton(ssp1), new Partition(1))
+    val cm1: ContainerModel = new ContainerModel("c1", 1, Collections.singletonMap(task1, tm1))
+
+    val cms: util.Map[String, ContainerModel] = new util.HashMap[String, ContainerModel]
+    cms.put(cm0.getProcessorId, cm0)
+    cms.put(cm1.getProcessorId, cm1)
+
+    val jobModel: JobModel = new JobModel(new MapConfig, cms, null)
+    val streamToTasks: Multimap[SystemStream, String] = TaskInstance.buildInputToTasks(jobModel)
+    assertEquals(streamToTasks.get(ssp0.getSystemStream).size, 2)
+    assertEquals(streamToTasks.get(ssp2.getSystemStream).size, 1)
+  }
 }
 
 class MockSystemAdmin extends SystemAdmin {

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java
----------------------------------------------------------------------
diff --git a/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java b/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java
index fb9bb56..de0d1da 100644
--- a/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java
+++ b/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java
@@ -36,6 +36,7 @@ import org.apache.commons.lang.Validate;
 import org.apache.samza.Partition;
 import org.apache.samza.SamzaException;
 import org.apache.samza.config.Config;
+import org.apache.samza.control.EndOfStreamManager;
 import org.apache.samza.metrics.Counter;
 import org.apache.samza.metrics.MetricsRegistry;
 import org.apache.samza.system.IncomingMessageEnvelope;
@@ -236,7 +237,7 @@ public class HdfsSystemConsumer extends BlockingEnvelopeMap {
       consumerMetrics.incNumEvents(systemStreamPartition);
       consumerMetrics.incTotalNumEvents();
     }
-    offerMessage(systemStreamPartition, IncomingMessageEnvelope.buildEndOfStreamEnvelope(systemStreamPartition));
+    offerMessage(systemStreamPartition, EndOfStreamManager.buildEndOfStreamEnvelope(systemStreamPartition));
     reader.close();
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-test/src/test/java/org/apache/samza/processor/TestStreamProcessorUtil.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/processor/TestStreamProcessorUtil.java b/samza-test/src/test/java/org/apache/samza/processor/TestStreamProcessorUtil.java
new file mode 100644
index 0000000..08e866e
--- /dev/null
+++ b/samza-test/src/test/java/org/apache/samza/processor/TestStreamProcessorUtil.java
@@ -0,0 +1,28 @@
+/*
+ * 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.processor;
+
+import org.apache.samza.container.SamzaContainer;
+
+public class TestStreamProcessorUtil {
+  public static SamzaContainer getContainer(StreamProcessor processor) {
+    return processor.getContainer();
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
new file mode 100644
index 0000000..26abb13
--- /dev/null
+++ b/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.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.test.controlmessages;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import org.apache.samza.application.StreamApplication;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.config.JobCoordinatorConfig;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.config.TaskConfig;
+import org.apache.samza.container.grouper.task.SingleContainerGrouperFactory;
+import org.apache.samza.runtime.LocalApplicationRunner;
+import org.apache.samza.standalone.PassthroughJobCoordinatorFactory;
+import org.apache.samza.test.controlmessages.TestData.PageView;
+import org.apache.samza.test.controlmessages.TestData.PageViewJsonSerdeFactory;
+import org.apache.samza.test.harness.AbstractIntegrationTestHarness;
+import org.apache.samza.test.util.ArraySystemFactory;
+import org.apache.samza.test.util.Base64Serializer;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+
+/**
+ * This test uses an array as a bounded input source, and does a partitionBy() and sink() after reading the input.
+ * It verifies the pipeline will stop and the number of output messages should equal to the input.
+ */
+public class EndOfStreamIntegrationTest extends AbstractIntegrationTestHarness {
+
+
+  private static final String[] PAGEKEYS = {"inbox", "home", "search", "pymk", "group", "job"};
+
+  @Test
+  public void testPipeline() throws  Exception {
+    Random random = new Random();
+    int count = 100;
+    PageView[] pageviews = new PageView[count];
+    for (int i = 0; i < count; i++) {
+      String pagekey = PAGEKEYS[random.nextInt(PAGEKEYS.length - 1)];
+      int memberId = random.nextInt(10);
+      pageviews[i] = new PageView(pagekey, memberId);
+    }
+
+    int partitionCount = 4;
+    Map<String, String> configs = new HashMap<>();
+    configs.put("systems.test.samza.factory", ArraySystemFactory.class.getName());
+    configs.put("streams.PageView.samza.system", "test");
+    configs.put("streams.PageView.source", Base64Serializer.serialize(pageviews));
+    configs.put("streams.PageView.partitionCount", String.valueOf(partitionCount));
+
+    configs.put(JobConfig.JOB_NAME(), "test-eos-job");
+    configs.put(JobConfig.PROCESSOR_ID(), "1");
+    configs.put(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, PassthroughJobCoordinatorFactory.class.getName());
+    configs.put(TaskConfig.GROUPER_FACTORY(), SingleContainerGrouperFactory.class.getName());
+
+    configs.put("systems.kafka.samza.factory", "org.apache.samza.system.kafka.KafkaSystemFactory");
+    configs.put("systems.kafka.producer.bootstrap.servers", bootstrapUrl());
+    configs.put("systems.kafka.consumer.zookeeper.connect", zkConnect());
+    configs.put("systems.kafka.samza.key.serde", "int");
+    configs.put("systems.kafka.samza.msg.serde", "json");
+    configs.put("systems.kafka.default.stream.replication.factor", "1");
+    configs.put("job.default.system", "kafka");
+
+    configs.put("serializers.registry.int.class", "org.apache.samza.serializers.IntegerSerdeFactory");
+    configs.put("serializers.registry.json.class", PageViewJsonSerdeFactory.class.getName());
+
+    final LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs));
+    List<PageView> received = new ArrayList<>();
+    final StreamApplication app = (streamGraph, cfg) -> {
+      streamGraph.getInputStream("PageView", (k, v) -> (PageView) v)
+        .partitionBy(PageView::getMemberId)
+        .sink((m, collector, coordinator) -> {
+            received.add(m);
+          });
+    };
+    runner.run(app);
+    runner.waitForFinish();
+
+    assertEquals(received.size(), count * partitionCount);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-test/src/test/java/org/apache/samza/test/controlmessages/TestData.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/controlmessages/TestData.java b/samza-test/src/test/java/org/apache/samza/test/controlmessages/TestData.java
new file mode 100644
index 0000000..8541b55
--- /dev/null
+++ b/samza-test/src/test/java/org/apache/samza/test/controlmessages/TestData.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.test.controlmessages;
+
+import java.io.Serializable;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.Config;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.serializers.SerdeFactory;
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.type.TypeReference;
+
+public class TestData {
+
+  public static class PageView implements Serializable {
+    @JsonProperty("pageKey")
+    final String pageKey;
+    @JsonProperty("memberId")
+    final int memberId;
+
+    @JsonProperty("pageKey")
+    public String getPageKey() {
+      return pageKey;
+    }
+
+    @JsonProperty("memberId")
+    public int getMemberId() {
+      return memberId;
+    }
+
+    @JsonCreator
+    public PageView(@JsonProperty("pageKey") String pageKey, @JsonProperty("memberId") int memberId) {
+      this.pageKey = pageKey;
+      this.memberId = memberId;
+    }
+  }
+
+  public static class PageViewJsonSerdeFactory implements SerdeFactory<PageView> {
+    @Override
+    public Serde<PageView> getSerde(String name, Config config) {
+      return new PageViewJsonSerde();
+    }
+  }
+
+  public static class PageViewJsonSerde implements Serde<PageView> {
+    ObjectMapper mapper = new ObjectMapper();
+
+    @Override
+    public PageView fromBytes(byte[] bytes) {
+      try {
+        return mapper.readValue(new String(bytes, "UTF-8"), new TypeReference<PageView>() { });
+      } catch (Exception e) {
+        throw new SamzaException(e);
+      }
+    }
+
+    @Override
+    public byte[] toBytes(PageView pv) {
+      try {
+        return mapper.writeValueAsString(pv).getBytes("UTF-8");
+      } catch (Exception e) {
+        throw new SamzaException(e);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java
new file mode 100644
index 0000000..58da8bd
--- /dev/null
+++ b/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java
@@ -0,0 +1,204 @@
+/*
+ * 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.test.controlmessages;
+
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.samza.Partition;
+import org.apache.samza.application.StreamApplication;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.config.JobCoordinatorConfig;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.config.TaskConfig;
+import org.apache.samza.container.SamzaContainer;
+import org.apache.samza.container.TaskInstance;
+import org.apache.samza.container.TaskName;
+import org.apache.samza.container.grouper.task.SingleContainerGrouperFactory;
+import org.apache.samza.control.EndOfStreamManager;
+import org.apache.samza.control.WatermarkManager;
+import org.apache.samza.metrics.MetricsRegistry;
+import org.apache.samza.operators.impl.InputOperatorImpl;
+import org.apache.samza.operators.impl.OperatorImpl;
+import org.apache.samza.operators.impl.OperatorImplGraph;
+import org.apache.samza.operators.impl.TestOperatorImpl;
+import org.apache.samza.operators.spec.OperatorSpec;
+import org.apache.samza.processor.StreamProcessor;
+import org.apache.samza.processor.TestStreamProcessorUtil;
+import org.apache.samza.runtime.LocalApplicationRunner;
+import org.apache.samza.runtime.TestLocalApplicationRunner;
+import org.apache.samza.serializers.IntegerSerdeFactory;
+import org.apache.samza.serializers.StringSerdeFactory;
+import org.apache.samza.standalone.PassthroughJobCoordinatorFactory;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.SystemAdmin;
+import org.apache.samza.system.SystemConsumer;
+import org.apache.samza.system.SystemFactory;
+import org.apache.samza.system.SystemProducer;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.task.AsyncStreamTaskAdapter;
+import org.apache.samza.task.StreamOperatorTask;
+import org.apache.samza.task.TestStreamOperatorTask;
+import org.apache.samza.test.controlmessages.TestData.PageView;
+import org.apache.samza.test.controlmessages.TestData.PageViewJsonSerdeFactory;
+import org.apache.samza.test.harness.AbstractIntegrationTestHarness;
+import org.apache.samza.test.util.SimpleSystemAdmin;
+import org.apache.samza.test.util.TestStreamConsumer;
+import org.junit.Test;
+import scala.collection.JavaConverters;
+
+import static org.junit.Assert.assertEquals;
+
+
+public class WatermarkIntegrationTest extends AbstractIntegrationTestHarness {
+
+  private static int offset = 1;
+  private static final String TEST_SYSTEM = "test";
+  private static final String TEST_STREAM = "PageView";
+  private static final int PARTITION_COUNT = 2;
+  private static final SystemStreamPartition SSP0 = new SystemStreamPartition(TEST_SYSTEM, TEST_STREAM, new Partition(0));
+  private static final SystemStreamPartition SSP1 = new SystemStreamPartition(TEST_SYSTEM, TEST_STREAM, new Partition(1));
+
+  private final static List<IncomingMessageEnvelope> TEST_DATA = new ArrayList<>();
+  static {
+    TEST_DATA.add(createIncomingMessage(new PageView("inbox", 1), SSP0));
+    TEST_DATA.add(createIncomingMessage(new PageView("home", 2), SSP1));
+    TEST_DATA.add(WatermarkManager.buildWatermarkEnvelope(1, SSP0));
+    TEST_DATA.add(WatermarkManager.buildWatermarkEnvelope(2, SSP1));
+    TEST_DATA.add(WatermarkManager.buildWatermarkEnvelope(4, SSP0));
+    TEST_DATA.add(WatermarkManager.buildWatermarkEnvelope(3, SSP1));
+    TEST_DATA.add(createIncomingMessage(new PageView("search", 3), SSP0));
+    TEST_DATA.add(createIncomingMessage(new PageView("pymk", 4), SSP1));
+    TEST_DATA.add(EndOfStreamManager.buildEndOfStreamEnvelope(SSP0));
+    TEST_DATA.add(EndOfStreamManager.buildEndOfStreamEnvelope(SSP1));
+  }
+
+  public final static class TestSystemFactory implements SystemFactory {
+    @Override
+    public SystemConsumer getConsumer(String systemName, Config config, MetricsRegistry registry) {
+      return new TestStreamConsumer(TEST_DATA);
+    }
+
+    @Override
+    public SystemProducer getProducer(String systemName, Config config, MetricsRegistry registry) {
+      return null;
+    }
+
+    @Override
+    public SystemAdmin getAdmin(String systemName, Config config) {
+      return new SimpleSystemAdmin(config);
+    }
+  }
+
+  private static IncomingMessageEnvelope createIncomingMessage(Object message, SystemStreamPartition ssp) {
+    return new IncomingMessageEnvelope(ssp, String.valueOf(offset++), "", message);
+  }
+
+  @Test
+  public void testWatermark() throws Exception {
+    Map<String, String> configs = new HashMap<>();
+    configs.put("systems.test.samza.factory", TestSystemFactory.class.getName());
+    configs.put("streams.PageView.samza.system", "test");
+    configs.put("streams.PageView.partitionCount", String.valueOf(PARTITION_COUNT));
+
+    configs.put(JobConfig.JOB_NAME(), "test-watermark-job");
+    configs.put(JobConfig.PROCESSOR_ID(), "1");
+    configs.put(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, PassthroughJobCoordinatorFactory.class.getName());
+    configs.put(TaskConfig.GROUPER_FACTORY(), SingleContainerGrouperFactory.class.getName());
+
+    configs.put("systems.kafka.samza.factory", "org.apache.samza.system.kafka.KafkaSystemFactory");
+    configs.put("systems.kafka.producer.bootstrap.servers", bootstrapUrl());
+    configs.put("systems.kafka.consumer.zookeeper.connect", zkConnect());
+    configs.put("systems.kafka.samza.key.serde", "int");
+    configs.put("systems.kafka.samza.msg.serde", "json");
+    configs.put("systems.kafka.default.stream.replication.factor", "1");
+    configs.put("job.default.system", "kafka");
+
+    configs.put("serializers.registry.int.class", IntegerSerdeFactory.class.getName());
+    configs.put("serializers.registry.string.class", StringSerdeFactory.class.getName());
+    configs.put("serializers.registry.json.class", PageViewJsonSerdeFactory.class.getName());
+
+    final LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs));
+    List<PageView> received = new ArrayList<>();
+    final StreamApplication app = (streamGraph, cfg) -> {
+      streamGraph.getInputStream("PageView", (k, v) -> (PageView) v)
+          .partitionBy(PageView::getMemberId)
+          .sink((m, collector, coordinator) -> {
+              received.add(m);
+            });
+    };
+    runner.run(app);
+    Map<String, StreamOperatorTask> tasks = getTaskOperationGraphs(runner);
+
+    runner.waitForFinish();
+
+    StreamOperatorTask task0 = tasks.get("Partition 0");
+    OperatorImplGraph graph = TestStreamOperatorTask.getOperatorImplGraph(task0);
+    OperatorImpl pb = getOperator(graph, OperatorSpec.OpCode.PARTITION_BY);
+    assertEquals(TestOperatorImpl.getInputWatermark(pb), 4);
+    assertEquals(TestOperatorImpl.getOutputWatermark(pb), 4);
+    OperatorImpl sink = getOperator(graph, OperatorSpec.OpCode.SINK);
+    assertEquals(TestOperatorImpl.getInputWatermark(sink), 3);
+    assertEquals(TestOperatorImpl.getOutputWatermark(sink), 3);
+
+    StreamOperatorTask task1 = tasks.get("Partition 1");
+    graph = TestStreamOperatorTask.getOperatorImplGraph(task1);
+    pb = getOperator(graph, OperatorSpec.OpCode.PARTITION_BY);
+    assertEquals(TestOperatorImpl.getInputWatermark(pb), 3);
+    assertEquals(TestOperatorImpl.getOutputWatermark(pb), 3);
+    sink = getOperator(graph, OperatorSpec.OpCode.SINK);
+    assertEquals(TestOperatorImpl.getInputWatermark(sink), 3);
+    assertEquals(TestOperatorImpl.getOutputWatermark(sink), 3);
+  }
+
+  Map<String, StreamOperatorTask> getTaskOperationGraphs(LocalApplicationRunner runner) throws Exception {
+    StreamProcessor processor = TestLocalApplicationRunner.getProcessors(runner).iterator().next();
+    SamzaContainer container = TestStreamProcessorUtil.getContainer(processor);
+    Map<TaskName, TaskInstance> taskInstances = JavaConverters.mapAsJavaMapConverter(container.getTaskInstances()).asJava();
+    Map<String, StreamOperatorTask> tasks = new HashMap<>();
+    for (Map.Entry<TaskName, TaskInstance> entry : taskInstances.entrySet()) {
+      AsyncStreamTaskAdapter adapter = (AsyncStreamTaskAdapter) entry.getValue().task();
+      Field field = AsyncStreamTaskAdapter.class.getDeclaredField("wrappedTask");
+      field.setAccessible(true);
+      StreamOperatorTask task = (StreamOperatorTask) field.get(adapter);
+      tasks.put(entry.getKey().getTaskName(), task);
+    }
+    return tasks;
+  }
+
+  OperatorImpl getOperator(OperatorImplGraph graph, OperatorSpec.OpCode opCode) {
+    for (InputOperatorImpl input : graph.getAllInputOperators()) {
+      Set<OperatorImpl> nextOps = TestOperatorImpl.getNextOperators(input);
+      while (!nextOps.isEmpty()) {
+        OperatorImpl op = nextOps.iterator().next();
+        if (TestOperatorImpl.getOpCode(op) == opCode) {
+          return op;
+        } else {
+          nextOps = TestOperatorImpl.getNextOperators(op);
+        }
+      }
+    }
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-test/src/test/java/org/apache/samza/test/util/ArraySystemConsumer.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/util/ArraySystemConsumer.java b/samza-test/src/test/java/org/apache/samza/test/util/ArraySystemConsumer.java
new file mode 100644
index 0000000..9b96216
--- /dev/null
+++ b/samza-test/src/test/java/org/apache/samza/test/util/ArraySystemConsumer.java
@@ -0,0 +1,83 @@
+/*
+ * 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.test.util;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.samza.config.Config;
+import org.apache.samza.control.EndOfStreamManager;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.SystemConsumer;
+import org.apache.samza.system.SystemStreamPartition;
+
+/**
+ * A simple implementation of array system consumer
+ */
+public class ArraySystemConsumer implements SystemConsumer {
+  boolean done = false;
+  private final Config config;
+
+  public ArraySystemConsumer(Config config) {
+    this.config = config;
+  }
+
+  @Override
+  public void start() {
+  }
+
+  @Override
+  public void stop() {
+  }
+
+  @Override
+  public void register(SystemStreamPartition systemStreamPartition, String s) {
+  }
+
+  @Override
+  public Map<SystemStreamPartition, List<IncomingMessageEnvelope>> poll(Set<SystemStreamPartition> set, long l) throws InterruptedException {
+    if (!done) {
+      Map<SystemStreamPartition, List<IncomingMessageEnvelope>> envelopeMap = new HashMap<>();
+      set.forEach(ssp -> {
+          List<IncomingMessageEnvelope> envelopes = Arrays.stream(getArrayObjects(ssp.getSystemStream().getStream(), config))
+              .map(object -> new IncomingMessageEnvelope(ssp, null, null, object)).collect(Collectors.toList());
+          envelopes.add(EndOfStreamManager.buildEndOfStreamEnvelope(ssp));
+          envelopeMap.put(ssp, envelopes);
+        });
+      done = true;
+      return envelopeMap;
+    } else {
+      return Collections.emptyMap();
+    }
+
+  }
+
+  private static Object[] getArrayObjects(String stream, Config config) {
+    try {
+      return Base64Serializer.deserialize(config.get("streams." + stream + ".source"), Object[].class);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-test/src/test/java/org/apache/samza/test/util/ArraySystemFactory.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/util/ArraySystemFactory.java b/samza-test/src/test/java/org/apache/samza/test/util/ArraySystemFactory.java
new file mode 100644
index 0000000..0632865
--- /dev/null
+++ b/samza-test/src/test/java/org/apache/samza/test/util/ArraySystemFactory.java
@@ -0,0 +1,50 @@
+/*
+ * 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.test.util;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.metrics.MetricsRegistry;
+import org.apache.samza.system.SystemAdmin;
+import org.apache.samza.system.SystemConsumer;
+import org.apache.samza.system.SystemFactory;
+import org.apache.samza.system.SystemProducer;
+
+
+/**
+ * System factory for the stream from an array
+ */
+public class ArraySystemFactory implements SystemFactory {
+
+  @Override
+  public SystemConsumer getConsumer(String systemName, Config config, MetricsRegistry metricsRegistry) {
+    return new ArraySystemConsumer(config);
+  }
+
+  @Override
+  public SystemProducer getProducer(String systemName, Config config, MetricsRegistry metricsRegistry) {
+    // no producer
+    return null;
+  }
+
+  @Override
+  public SystemAdmin getAdmin(String systemName, Config config) {
+    return new SimpleSystemAdmin(config);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-test/src/test/java/org/apache/samza/test/util/Base64Serializer.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/util/Base64Serializer.java b/samza-test/src/test/java/org/apache/samza/test/util/Base64Serializer.java
new file mode 100644
index 0000000..1a17a3d
--- /dev/null
+++ b/samza-test/src/test/java/org/apache/samza/test/util/Base64Serializer.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.test.util;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.util.Base64;
+
+
+public class Base64Serializer {
+  private Base64Serializer() {}
+
+  public static String serializeUnchecked(Serializable serializable) {
+    try {
+      return serialize(serializable);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public static String serialize(Serializable serializable) throws IOException {
+    final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    final ObjectOutputStream oos = new ObjectOutputStream(baos);
+    oos.writeObject(serializable);
+    oos.close();
+    return Base64.getEncoder().encodeToString(baos.toByteArray());
+  }
+
+  public static <T> T deserializeUnchecked(String serialized, Class<T> klass) {
+    try {
+      return deserialize(serialized, klass);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public static <T> T deserialize(String serialized, Class<T> klass) throws IOException, ClassNotFoundException {
+    final byte[] bytes = Base64.getDecoder().decode(serialized);
+    final ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(bytes));
+    @SuppressWarnings("unchecked")
+    T object = (T) ois.readObject();
+    ois.close();
+    return object;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-test/src/test/java/org/apache/samza/test/util/SimpleSystemAdmin.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/util/SimpleSystemAdmin.java b/samza-test/src/test/java/org/apache/samza/test/util/SimpleSystemAdmin.java
new file mode 100644
index 0000000..41f01c5
--- /dev/null
+++ b/samza-test/src/test/java/org/apache/samza/test/util/SimpleSystemAdmin.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.test.util;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.samza.Partition;
+import org.apache.samza.config.Config;
+import org.apache.samza.system.SystemAdmin;
+import org.apache.samza.system.SystemStreamMetadata;
+import org.apache.samza.system.SystemStreamPartition;
+
+
+/**
+ * A dummy system admin
+ */
+public class SimpleSystemAdmin implements SystemAdmin {
+  private final Config config;
+
+  public SimpleSystemAdmin(Config config) {
+    this.config = config;
+  }
+
+  @Override
+  public Map<SystemStreamPartition, String> getOffsetsAfter(Map<SystemStreamPartition, String> offsets) {
+    return offsets.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, null));
+  }
+
+  @Override
+  public Map<String, SystemStreamMetadata> getSystemStreamMetadata(Set<String> streamNames) {
+    return streamNames.stream()
+        .collect(Collectors.toMap(
+            Function.<String>identity(),
+            streamName -> {
+            Map<Partition, SystemStreamMetadata.SystemStreamPartitionMetadata> metadataMap = new HashMap<>();
+            int partitionCount = config.getInt("streams." + streamName + ".partitionCount", 1);
+            for (int i = 0; i < partitionCount; i++) {
+              metadataMap.put(new Partition(i), new SystemStreamMetadata.SystemStreamPartitionMetadata(null, null, null));
+            }
+            return new SystemStreamMetadata(streamName, metadataMap);
+          }));
+  }
+
+  @Override
+  public void createChangelogStream(String streamName, int numOfPartitions) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void validateChangelogStream(String streamName, int numOfPartitions) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void createCoordinatorStream(String streamName) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Integer offsetComparator(String offset1, String offset2) {
+    if (offset1 == null) {
+      return offset2 == null ? 0 : -1;
+    } else if (offset2 == null) {
+      return 1;
+    }
+    return offset1.compareTo(offset2);
+  }
+}
+


[09/16] samza git commit: SAMZA-1406: Fix potential orphaned containers problem in stand alone

Posted by xi...@apache.org.
SAMZA-1406: Fix potential orphaned containers problem in stand alone


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

Branch: refs/heads/master
Commit: c45c7747ae371eee11e2f41dd4e32a53b12c6c91
Parents: ebce13e
Author: Shanthoosh Venkataraman <sv...@linkedin.com>
Authored: Thu Sep 14 21:17:36 2017 -0700
Committer: navina <na...@apache.org>
Committed: Thu Sep 14 21:18:55 2017 -0700

----------------------------------------------------------------------
 .../samza/zk/ScheduleAfterDebounceTime.java     | 172 +++++++++++++------
 .../org/apache/samza/zk/ZkJobCoordinator.java   |  26 ++-
 .../samza/zk/TestScheduleAfterDebounceTime.java |  74 +++++++-
 3 files changed, 204 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/c45c7747/samza-core/src/main/java/org/apache/samza/zk/ScheduleAfterDebounceTime.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ScheduleAfterDebounceTime.java b/samza-core/src/main/java/org/apache/samza/zk/ScheduleAfterDebounceTime.java
index 6174063..3a7dca9 100644
--- a/samza-core/src/main/java/org/apache/samza/zk/ScheduleAfterDebounceTime.java
+++ b/samza-core/src/main/java/org/apache/samza/zk/ScheduleAfterDebounceTime.java
@@ -20,99 +20,157 @@
 package org.apache.samza.zk;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-import java.util.HashMap;
-import java.util.Map;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
-
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
  * This class allows scheduling a Runnable actions after some debounce time.
  * When the same action is scheduled it needs to cancel the previous one. To accomplish that we keep the previous
- * future in a map, keyed by the action name. Here we predefine some actions, which are used in the
- * ZK based standalone app.
+ * future in a map, keyed by the action name.
  */
 public class ScheduleAfterDebounceTime {
-  public static final Logger LOG = LoggerFactory.getLogger(ScheduleAfterDebounceTime.class);
-  public static final long TIMEOUT_MS = 1000 * 10; // timeout to wait for a task to complete
+  private static final Logger LOG = LoggerFactory.getLogger(ScheduleAfterDebounceTime.class);
+  private static final String DEBOUNCE_THREAD_NAME_FORMAT = "debounce-thread-%d";
 
-  // Here we predefine some actions which are used in the ZK based standalone app.
-  // Action name when the JobModel version changes
-  public static final String JOB_MODEL_VERSION_CHANGE = "JobModelVersionChange";
-
-  // Action name when the Processor membership changes
-  public static final String ON_PROCESSOR_CHANGE = "OnProcessorChange";
+  // timeout to wait for a task to complete.
+  private static final int TIMEOUT_MS = 1000 * 10;
 
   /**
-   *
-   * cleanup process is started after every new job model generation is complete.
-   * It deletes old versions of job model and the barrier.
-   * How many to delete (or to leave) is controlled by @see org.apache.samza.zk.ZkJobCoordinator#NUM_VERSIONS_TO_LEAVE.
-   **/
-  public static final String ON_ZK_CLEANUP = "OnCleanUp";
+   * {@link ScheduledTaskCallback} associated with the scheduler. OnError method of the
+   * callback will be invoked on first scheduled task failure.
+   */
+  private Optional<ScheduledTaskCallback> scheduledTaskCallback;
 
-  private final ScheduledTaskFailureCallback scheduledTaskFailureCallback;
+  // Responsible for scheduling delayed actions.
+  private final ScheduledExecutorService scheduledExecutorService;
 
-  private final ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(
-      new ThreadFactoryBuilder().setNameFormat("debounce-thread-%d").setDaemon(true).build());
+  /**
+   * A map from actionName to {@link ScheduledFuture} of task scheduled for execution.
+   */
   private final Map<String, ScheduledFuture> futureHandles = new HashMap<>();
 
-  // Ideally, this should be only used for testing. But ZkBarrierForVersionUpgrades uses it. This needs to be fixed.
-  // TODO: Timer shouldn't be passed around the components. It should be associated with the JC or the caller of
-  // coordinationUtils.
   public ScheduleAfterDebounceTime() {
-    this.scheduledTaskFailureCallback = null;
+    ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat(DEBOUNCE_THREAD_NAME_FORMAT).setDaemon(true).build();
+    this.scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(threadFactory);
+  }
+
+  public void setScheduledTaskCallback(ScheduledTaskCallback scheduledTaskCallback) {
+    this.scheduledTaskCallback = Optional.ofNullable(scheduledTaskCallback);
   }
 
-  public ScheduleAfterDebounceTime(ScheduledTaskFailureCallback errorScheduledTaskFailureCallback) {
-    this.scheduledTaskFailureCallback = errorScheduledTaskFailureCallback;
+  /**
+   * Performs the following operations in sequential order.
+   * <ul>
+   *    <li> Makes best effort to cancel any existing task in task queue associated with the action.</li>
+   *    <li> Schedules the incoming action for later execution and records its future.</li>
+   * </ul>
+   *
+   * @param actionName the name of scheduleable action.
+   * @param delayInMillis the time from now to delay execution.
+   * @param runnable the action to execute.
+   */
+  public synchronized void scheduleAfterDebounceTime(String actionName, long delayInMillis, Runnable runnable) {
+    // 1. Try to cancel any existing scheduled task associated with the action.
+    tryCancelScheduledAction(actionName);
+
+    // 2. Schedule the action.
+    ScheduledFuture scheduledFuture = scheduledExecutorService.schedule(getScheduleableAction(actionName, runnable), delayInMillis, TimeUnit.MILLISECONDS);
+
+    LOG.info("Scheduled action: {} to run after: {} milliseconds.", actionName, delayInMillis);
+    futureHandles.put(actionName, scheduledFuture);
   }
 
-  synchronized public void scheduleAfterDebounceTime(String actionName, long debounceTimeMs, Runnable runnable) {
-    // check if this action has been scheduled already
-    ScheduledFuture sf = futureHandles.get(actionName);
-    if (sf != null && !sf.isDone()) {
-      LOG.info("cancel future for " + actionName);
-      // attempt to cancel
-      if (!sf.cancel(false)) {
+  /**
+   * Stops the scheduler. After this invocation no further schedule calls will be accepted
+   * and all pending enqueued tasks will be cancelled.
+   */
+  public synchronized void stopScheduler() {
+    scheduledExecutorService.shutdownNow();
+
+    // Clear the existing future handles.
+    futureHandles.clear();
+  }
+
+  /**
+   * Tries to cancel the task that belongs to {@code actionName} submitted to the queue.
+   *
+   * @param actionName the name of action to cancel.
+   */
+  private void tryCancelScheduledAction(String actionName) {
+    ScheduledFuture scheduledFuture = futureHandles.get(actionName);
+    if (scheduledFuture != null && !scheduledFuture.isDone()) {
+      LOG.info("Attempting to cancel the future of action: {}", actionName);
+      // Attempt to cancel
+      if (!scheduledFuture.cancel(false)) {
         try {
-          sf.get(TIMEOUT_MS, TimeUnit.MILLISECONDS);
+          scheduledFuture.get(TIMEOUT_MS, TimeUnit.MILLISECONDS);
         } catch (Exception e) {
           // we ignore the exception
-          LOG.warn("cancel for action " + actionName + " failed with ", e);
+          LOG.warn("Cancelling the future of action: {} failed.", actionName, e);
         }
       }
       futureHandles.remove(actionName);
     }
-    // schedule a new task
-    sf = scheduledExecutorService.schedule(() -> {
-        try {
-          runnable.run();
-          LOG.debug(actionName + " completed successfully.");
-        } catch (Throwable t) {
-          LOG.error(actionName + " threw an exception.", t);
-          if (scheduledTaskFailureCallback != null) {
-            scheduledTaskFailureCallback.onError(t);
-          }
+  }
+
+  /**
+   * Decorate the executable action with exception handlers to facilitate cleanup on failures.
+   *
+   * @param actionName the name of the scheduleable action.
+   * @param runnable the action to execute.
+   * @return the executable action decorated with exception handlers.
+   */
+  private Runnable getScheduleableAction(String actionName, Runnable runnable) {
+    return () -> {
+      try {
+        runnable.run();
+        /*
+         * Expects all run() implementations <b>not to swallow the interrupts.</b>
+         * This thread is interrupted from an external source(mostly executor service) to die.
+         */
+        if (Thread.currentThread().isInterrupted()) {
+          LOG.warn("Action: {} is interrupted.", actionName);
+          doCleanUpOnTaskException(new InterruptedException());
+        } else {
+          LOG.debug("Action: {} completed successfully.", actionName);
         }
-      },
-     debounceTimeMs,
-     TimeUnit.MILLISECONDS);
-    LOG.info("scheduled " + actionName + " in " + debounceTimeMs);
-    futureHandles.put(actionName, sf);
+      } catch (Exception exception) {
+        LOG.error("Execution of action: {} failed.", actionName, exception);
+        doCleanUpOnTaskException(exception);
+      }
+    };
   }
 
-  public void stopScheduler() {
-    // shutdown executor service
-    scheduledExecutorService.shutdown();
+  /**
+   * Handler method to invoke on a exception during an scheduled task execution and which
+   * the following operations in sequential order.
+   * <ul>
+   *   <li> Stop the scheduler. If the task execution fails or a task is interrupted, scheduler will not accept/execute any new tasks.</li>
+   *   <li> Invokes the onError handler method if taskCallback is defined.</li>
+   * </ul>
+   *
+   * @param exception the exception happened during task execution.
+   */
+  private void doCleanUpOnTaskException(Exception exception) {
+    stopScheduler();
+
+    scheduledTaskCallback.ifPresent(callback -> callback.onError(exception));
   }
 
-  interface ScheduledTaskFailureCallback {
+  /**
+   * A ScheduledTaskCallback::onError() is invoked on first occurrence of exception
+   * when executing a task. Provides plausible hook for handling failures
+   * in an asynchronous scheduled task execution.
+   */
+  interface ScheduledTaskCallback {
     void onError(Throwable throwable);
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/c45c7747/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinator.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinator.java b/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinator.java
index 9f64b3a..2b8349c 100644
--- a/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinator.java
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinator.java
@@ -61,6 +61,19 @@ public class ZkJobCoordinator implements JobCoordinator, ZkControllerListener {
   private static final int METADATA_CACHE_TTL_MS = 5000;
   private static final int NUM_VERSIONS_TO_LEAVE = 10;
 
+  // Action name when the JobModel version changes
+  private static final String JOB_MODEL_VERSION_CHANGE = "JobModelVersionChange";
+
+  // Action name when the Processor membership changes
+  private static final String ON_PROCESSOR_CHANGE = "OnProcessorChange";
+
+  /**
+   * Cleanup process is started after every new job model generation is complete.
+   * It deletes old versions of job model and the barrier.
+   * How many to delete (or to leave) is controlled by @see org.apache.samza.zk.ZkJobCoordinator#NUM_VERSIONS_TO_LEAVE.
+   **/
+  private static final String ON_ZK_CLEANUP = "OnCleanUp";
+
   private final ZkUtils zkUtils;
   private final String processorId;
   private final ZkController zkController;
@@ -95,7 +108,8 @@ public class ZkJobCoordinator implements JobCoordinator, ZkControllerListener {
         new ZkBarrierListenerImpl());
     this.debounceTimeMs = new JobConfig(config).getDebounceTimeMs();
     this.reporters = MetricsReporterLoader.getMetricsReporters(new MetricsConfig(config), processorId);
-    debounceTimer = new ScheduleAfterDebounceTime(throwable -> {
+    debounceTimer = new ScheduleAfterDebounceTime();
+    debounceTimer.setScheduledTaskCallback(throwable -> {
         LOG.error("Received exception from in JobCoordinator Processing!", throwable);
         stop();
       });
@@ -157,7 +171,7 @@ public class ZkJobCoordinator implements JobCoordinator, ZkControllerListener {
   @Override
   public void onProcessorChange(List<String> processors) {
     LOG.info("ZkJobCoordinator::onProcessorChange - list of processors changed! List size=" + processors.size());
-    debounceTimer.scheduleAfterDebounceTime(ScheduleAfterDebounceTime.ON_PROCESSOR_CHANGE, debounceTimeMs,
+    debounceTimer.scheduleAfterDebounceTime(ON_PROCESSOR_CHANGE, debounceTimeMs,
         () -> doOnProcessorChange(processors));
   }
 
@@ -195,12 +209,12 @@ public class ZkJobCoordinator implements JobCoordinator, ZkControllerListener {
 
     LOG.info("pid=" + processorId + "Published new Job Model. Version = " + nextJMVersion);
 
-    debounceTimer.scheduleAfterDebounceTime(ScheduleAfterDebounceTime.ON_ZK_CLEANUP, 0, () -> zkUtils.cleanupZK(NUM_VERSIONS_TO_LEAVE));
+    debounceTimer.scheduleAfterDebounceTime(ON_ZK_CLEANUP, 0, () -> zkUtils.cleanupZK(NUM_VERSIONS_TO_LEAVE));
   }
 
   @Override
   public void onNewJobModelAvailable(final String version) {
-    debounceTimer.scheduleAfterDebounceTime(ScheduleAfterDebounceTime.JOB_MODEL_VERSION_CHANGE, 0, () ->
+    debounceTimer.scheduleAfterDebounceTime(JOB_MODEL_VERSION_CHANGE, 0, () ->
       {
         LOG.info("pid=" + processorId + "new JobModel available");
         // get the new job model from ZK
@@ -273,7 +287,7 @@ public class ZkJobCoordinator implements JobCoordinator, ZkControllerListener {
       LOG.info("ZkJobCoordinator::onBecomeLeader - I became the leader!");
       metrics.isLeader.set(true);
       zkController.subscribeToProcessorChange();
-      debounceTimer.scheduleAfterDebounceTime(ScheduleAfterDebounceTime.ON_PROCESSOR_CHANGE, debounceTimeMs, () ->
+      debounceTimer.scheduleAfterDebounceTime(ON_PROCESSOR_CHANGE, debounceTimeMs, () ->
         {
           // actual actions to do are the same as onProcessorChange
           doOnProcessorChange(new ArrayList<>());
@@ -312,7 +326,7 @@ public class ZkJobCoordinator implements JobCoordinator, ZkControllerListener {
           LOG.warn("Barrier for version " + version + " timed out.");
           if (zkController.isLeader()) {
             LOG.info("Leader will schedule a new job model generation");
-            debounceTimer.scheduleAfterDebounceTime(ScheduleAfterDebounceTime.ON_PROCESSOR_CHANGE, debounceTimeMs, () ->
+            debounceTimer.scheduleAfterDebounceTime(ON_PROCESSOR_CHANGE, debounceTimeMs, () ->
               {
                 // actual actions to do are the same as onProcessorChange
                 doOnProcessorChange(new ArrayList<>());

http://git-wip-us.apache.org/repos/asf/samza/blob/c45c7747/samza-core/src/test/java/org/apache/samza/zk/TestScheduleAfterDebounceTime.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/zk/TestScheduleAfterDebounceTime.java b/samza-core/src/test/java/org/apache/samza/zk/TestScheduleAfterDebounceTime.java
index d3152be..a681767 100644
--- a/samza-core/src/test/java/org/apache/samza/zk/TestScheduleAfterDebounceTime.java
+++ b/samza-core/src/test/java/org/apache/samza/zk/TestScheduleAfterDebounceTime.java
@@ -19,15 +19,23 @@
 
 package org.apache.samza.zk;
 
-import org.junit.Assert;
-import org.junit.Test;
-
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.rules.Timeout;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestScheduleAfterDebounceTime {
+  private static final Logger LOG = LoggerFactory.getLogger(TestScheduleAfterDebounceTime.class);
+
   private static final long WAIT_TIME = 500;
 
+  @Rule
+  public Timeout testTimeOutInSeconds = new Timeout(10, TimeUnit.SECONDS);
+
   class TestObj {
     private volatile int i = 0;
     public void inc() {
@@ -91,8 +99,10 @@ public class TestScheduleAfterDebounceTime {
   @Test
   public void testRunnableWithExceptionInvokesCallback() throws InterruptedException {
     final CountDownLatch latch = new CountDownLatch(1);
-    ScheduleAfterDebounceTime scheduledQueue = new ScheduleAfterDebounceTime(e -> {
-        Assert.assertEquals(RuntimeException.class, e.getClass());
+    final Throwable[] taskCallbackException = new Exception[1];
+    ScheduleAfterDebounceTime scheduledQueue = new ScheduleAfterDebounceTime();
+    scheduledQueue.setScheduledTaskCallback(throwable -> {
+        taskCallbackException[0] = throwable;
         latch.countDown();
       });
 
@@ -107,6 +117,60 @@ public class TestScheduleAfterDebounceTime {
     boolean result = latch.await(5 * WAIT_TIME, TimeUnit.MILLISECONDS);
     Assert.assertTrue("Latch timed-out.", result);
     Assert.assertEquals(0, testObj.get());
+    Assert.assertEquals(RuntimeException.class, taskCallbackException[0].getClass());
+    scheduledQueue.stopScheduler();
+  }
+
+  /**
+   * Validates if the interrupted exception triggered by ExecutorService is handled by ScheduleAfterDebounceTime.
+   */
+  @Test
+  public void testStopSchedulerInvokesRegisteredCallback() throws InterruptedException {
+    final CountDownLatch hasTaskCallbackCompleted = new CountDownLatch(1);
+    final CountDownLatch hasThreadStarted = new CountDownLatch(1);
+    final CountDownLatch isSchedulerShutdownTriggered = new CountDownLatch(1);
+
+    /**
+     * Declaring this as an array to record the value inside the lambda.
+     */
+    final Throwable[] taskCallbackException = new Exception[1];
+
+    ScheduleAfterDebounceTime scheduledQueue = new ScheduleAfterDebounceTime();
+    scheduledQueue.setScheduledTaskCallback(throwable -> {
+      /**
+       * Assertion failures in callback doesn't fail the test.
+       * Record the received exception here and assert outside
+       * the callback.
+       */
+        taskCallbackException[0] = throwable;
+        hasTaskCallbackCompleted.countDown();
+      });
+
+    scheduledQueue.scheduleAfterDebounceTime("TEST1", WAIT_TIME , () -> {
+        hasThreadStarted.countDown();
+        try {
+          LOG.debug("Waiting for the scheduler shutdown trigger.");
+          isSchedulerShutdownTriggered.await();
+        } catch (InterruptedException e) {
+          /**
+           * Don't swallow the exception and restore the interrupt status.
+           * Expect the ScheduleDebounceTime to handle this interrupt
+           * and invoke ScheduledTaskCallback.
+           */
+          Thread.currentThread().interrupt();
+        }
+      });
+
+    // Wait for the task to run.
+    hasThreadStarted.await();
+
+    // Shutdown the scheduler and update relevant state.
     scheduledQueue.stopScheduler();
+    isSchedulerShutdownTriggered.countDown();
+
+    hasTaskCallbackCompleted.await();
+
+    // Assert on exception thrown.
+    Assert.assertEquals(InterruptedException.class, taskCallbackException[0].getClass());
   }
 }


[08/16] samza git commit: Fix some integration tests after merging from master

Posted by xi...@apache.org.
Fix some integration tests after merging from master


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

Branch: refs/heads/master
Commit: ebce13e7461d80f2331b4005cb287719cdad88ad
Parents: 1701ea8
Author: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>
Authored: Tue Sep 12 14:16:27 2017 -0700
Committer: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>
Committed: Tue Sep 12 14:16:27 2017 -0700

----------------------------------------------------------------------
 .../PassthroughCoordinationUtilsFactory.java    | 30 ++++++++++++++++++++
 .../EndOfStreamIntegrationTest.java             |  3 +-
 .../WatermarkIntegrationTest.java               |  2 ++
 3 files changed, 34 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/ebce13e7/samza-core/src/main/java/org/apache/samza/standalone/PassthroughCoordinationUtilsFactory.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/standalone/PassthroughCoordinationUtilsFactory.java b/samza-core/src/main/java/org/apache/samza/standalone/PassthroughCoordinationUtilsFactory.java
new file mode 100644
index 0000000..ea38c88
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/standalone/PassthroughCoordinationUtilsFactory.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.standalone;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.coordinator.CoordinationUtils;
+import org.apache.samza.coordinator.CoordinationUtilsFactory;
+
+public class PassthroughCoordinationUtilsFactory implements CoordinationUtilsFactory {
+  @Override
+  public CoordinationUtils getCoordinationUtils(String groupId, String participantId, Config updatedConfig) {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/ebce13e7/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
index 26abb13..f313348 100644
--- a/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
+++ b/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
@@ -31,6 +31,7 @@ import org.apache.samza.config.MapConfig;
 import org.apache.samza.config.TaskConfig;
 import org.apache.samza.container.grouper.task.SingleContainerGrouperFactory;
 import org.apache.samza.runtime.LocalApplicationRunner;
+import org.apache.samza.standalone.PassthroughCoordinationUtilsFactory;
 import org.apache.samza.standalone.PassthroughJobCoordinatorFactory;
 import org.apache.samza.test.controlmessages.TestData.PageView;
 import org.apache.samza.test.controlmessages.TestData.PageViewJsonSerdeFactory;
@@ -48,7 +49,6 @@ import static org.junit.Assert.assertEquals;
  */
 public class EndOfStreamIntegrationTest extends AbstractIntegrationTestHarness {
 
-
   private static final String[] PAGEKEYS = {"inbox", "home", "search", "pymk", "group", "job"};
 
   @Test
@@ -71,6 +71,7 @@ public class EndOfStreamIntegrationTest extends AbstractIntegrationTestHarness {
 
     configs.put(JobConfig.JOB_NAME(), "test-eos-job");
     configs.put(JobConfig.PROCESSOR_ID(), "1");
+    configs.put(JobCoordinatorConfig.JOB_COORDINATION_UTILS_FACTORY, PassthroughCoordinationUtilsFactory.class.getName());
     configs.put(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, PassthroughJobCoordinatorFactory.class.getName());
     configs.put(TaskConfig.GROUPER_FACTORY(), SingleContainerGrouperFactory.class.getName());
 

http://git-wip-us.apache.org/repos/asf/samza/blob/ebce13e7/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java
index 58da8bd..2eb72fc 100644
--- a/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java
+++ b/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java
@@ -50,6 +50,7 @@ import org.apache.samza.runtime.LocalApplicationRunner;
 import org.apache.samza.runtime.TestLocalApplicationRunner;
 import org.apache.samza.serializers.IntegerSerdeFactory;
 import org.apache.samza.serializers.StringSerdeFactory;
+import org.apache.samza.standalone.PassthroughCoordinationUtilsFactory;
 import org.apache.samza.standalone.PassthroughJobCoordinatorFactory;
 import org.apache.samza.system.IncomingMessageEnvelope;
 import org.apache.samza.system.SystemAdmin;
@@ -124,6 +125,7 @@ public class WatermarkIntegrationTest extends AbstractIntegrationTestHarness {
 
     configs.put(JobConfig.JOB_NAME(), "test-watermark-job");
     configs.put(JobConfig.PROCESSOR_ID(), "1");
+    configs.put(JobCoordinatorConfig.JOB_COORDINATION_UTILS_FACTORY, PassthroughCoordinationUtilsFactory.class.getName());
     configs.put(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, PassthroughJobCoordinatorFactory.class.getName());
     configs.put(TaskConfig.GROUPER_FACTORY(), SingleContainerGrouperFactory.class.getName());
 


[10/16] samza git commit: SAMZA-1417: Clear and recreate intermediate and metadata streams for batch processing

Posted by xi...@apache.org.
SAMZA-1417: Clear and recreate intermediate and metadata streams for batch processing

For each run of a batch application, we need to clear the internal streams from the previous run and recreate new ones. This patch introduces the following:
1) bounded flag in StreamSpec
2) app.mode (BATCH/STREAM) in the application config. An application is a batch app iff all the input streams are bounded.
3) app.runId and use it to generate the internal topics for each run.

run.id generation is not addressed in this pr. There will be another patch to resolve it for both yarn and standalone. For now, this patch only works for yarn.

Author: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>

Reviewers: Jake Maes <jm...@apache.org>

Closes #297 from xinyuiscool/SAMZA-1417


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

Branch: refs/heads/master
Commit: 475414884ef386af328ed7a07d59f6986713c283
Parents: c45c774
Author: Xinyu Liu <xi...@gmail.com>
Authored: Mon Sep 25 14:52:42 2017 -0700
Committer: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>
Committed: Mon Sep 25 14:52:42 2017 -0700

----------------------------------------------------------------------
 .../versioned/jobs/configuration-table.html     |  32 ++++
 .../samza/checkpoint/CheckpointManager.java     |   4 +
 .../org/apache/samza/system/StreamSpec.java     |  37 ++++-
 .../apache/samza/config/ApplicationConfig.java  |  15 ++
 .../apache/samza/config/JavaStorageConfig.java  |   3 +
 .../apache/samza/execution/ExecutionPlan.java   |   7 +
 .../samza/execution/ExecutionPlanner.java       |  15 ++
 .../org/apache/samza/execution/JobGraph.java    |   2 +-
 .../org/apache/samza/execution/JobNode.java     |  16 +-
 .../org/apache/samza/execution/StreamEdge.java  |  47 +++++-
 .../apache/samza/execution/StreamManager.java   |  76 +++++++++
 .../runtime/AbstractApplicationRunner.java      |  42 ++++-
 .../samza/runtime/RemoteApplicationRunner.java  |  27 ++-
 .../org/apache/samza/zk/ZkJobCoordinator.java   |   1 -
 .../org/apache/samza/config/StreamConfig.scala  |  11 +-
 .../samza/coordinator/JobModelManager.scala     |   2 +-
 .../samza/execution/TestExecutionPlanner.java   |   6 +-
 .../apache/samza/execution/TestStreamEdge.java  |  84 ++++++++++
 .../samza/execution/TestStreamManager.java      | 147 +++++++++++++++++
 .../runtime/TestLocalApplicationRunner.java     | 164 +++++--------------
 .../samza/system/kafka/KafkaStreamSpec.java     |   2 +-
 .../kafka/KafkaCheckpointManager.scala          |  19 ++-
 .../kafka/KafkaCheckpointManagerFactory.scala   |  14 +-
 .../samza/system/kafka/KafkaSystemAdmin.scala   |   2 +
 .../scala/org/apache/samza/util/KafkaUtil.scala |  12 +-
 .../samza/system/kafka/TestKafkaStreamSpec.java |   2 +-
 26 files changed, 605 insertions(+), 184 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/docs/learn/documentation/versioned/jobs/configuration-table.html
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/versioned/jobs/configuration-table.html b/docs/learn/documentation/versioned/jobs/configuration-table.html
index 9b4e279..f4d87f6 100644
--- a/docs/learn/documentation/versioned/jobs/configuration-table.html
+++ b/docs/learn/documentation/versioned/jobs/configuration-table.html
@@ -116,6 +116,38 @@
         <table>
             <tbody>
                 <tr><th>Name</th><th>Default</th><th>Description</th></tr>
+
+                <tr>
+                    <th colspan="3" class="section" id="application">Samza application configuration</th>
+                </tr>
+                <tr>
+                    <td class="property" id="app-name">app.name</td>
+                    <td class="default"></td>
+                    <td class="description">
+                        <strong>Required:</strong> The name of your application.
+                    </td>
+                </tr>
+
+                <tr>
+                    <td class="property" id="app-id">app.id</td>
+                    <td class="default">1</td>
+                    <td class="description">
+                        If you run several instances of your application at the same time, you need to give each instance a
+                        different <code>app.id</code>. This is important, since otherwise the applications will overwrite each
+                        others' checkpoints, and perhaps interfere with each other in other ways.
+                    </td>
+                </tr>
+
+                <tr>
+                    <td class="property" id="app-class">app.class</td>
+                    <td class="default"></td>
+                    <td class="description">
+                        <strong>Required:</strong> The application to run. The value is a fully-qualified Java classname,
+                        which must implement <a href="../api/javadocs/org/apache/samza/application/StreamApplication.html">StreamApplication</a>.
+                        A StreamApplication describes as a series of transformations on the streams.
+                    </td>
+                </tr>
+
                 <tr>
                     <th colspan="3" class="section" id="job"><a href="configuration.html">Samza job configuration</a></th>
                 </tr>

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointManager.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointManager.java b/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointManager.java
index dc14beb..10f166c 100644
--- a/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointManager.java
+++ b/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointManager.java
@@ -50,4 +50,8 @@ public interface CheckpointManager {
 
   void stop();
 
+  /**
+   * Clear the checkpoints in the checkpoint stream.
+   */
+  default void clearCheckpoints() { };
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java b/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
index 384fecc..8d7401a 100644
--- a/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
+++ b/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
@@ -43,6 +43,9 @@ public class StreamSpec {
   // Internal coordinator stream id. It is used for creating coordinator StreamSpec.
   private static final String COORDINATOR_STREAM_ID = "samza-internal-coordinator-stream-id";
 
+  // Internal checkpoint stream id. It is used for creating checkpoint StreamSpec.
+  private static final String CHECKPOINT_STREAM_ID = "samza-internal-checkpoint-stream-id";
+
   /**
    * Unique identifier for the stream in a Samza application.
    * This identifier is used as a key for stream properties in the
@@ -69,6 +72,11 @@ public class StreamSpec {
   private final int partitionCount;
 
   /**
+   * Bounded or unbounded stream
+   */
+  private final boolean isBounded;
+
+  /**
    * A set of all system-specific configurations for the stream.
    */
   private final Map<String, String> config;
@@ -86,7 +94,7 @@ public class StreamSpec {
    *                      Samza System abstraction. See {@link SystemFactory}
    */
   public StreamSpec(String id, String physicalName, String systemName) {
-    this(id, physicalName, systemName, DEFAULT_PARTITION_COUNT, Collections.emptyMap());
+    this(id, physicalName, systemName, DEFAULT_PARTITION_COUNT, false, Collections.emptyMap());
   }
 
   /**
@@ -105,7 +113,7 @@ public class StreamSpec {
    * @param partitionCount  The number of partitionts for the stream. A value of {@code 1} indicates unpartitioned.
    */
   public StreamSpec(String id, String physicalName, String systemName, int partitionCount) {
-    this(id, physicalName, systemName, partitionCount, Collections.emptyMap());
+    this(id, physicalName, systemName, partitionCount, false, Collections.emptyMap());
   }
 
   /**
@@ -120,10 +128,12 @@ public class StreamSpec {
    * @param systemName    The System name on which this stream will exist. Corresponds to a named implementation of the
    *                      Samza System abstraction. See {@link SystemFactory}
    *
+   * @param isBounded     The stream is bounded or not.
+   *
    * @param config        A map of properties for the stream. These may be System-specfic.
    */
-  public StreamSpec(String id, String physicalName, String systemName, Map<String, String> config) {
-    this(id, physicalName, systemName, DEFAULT_PARTITION_COUNT, config);
+  public StreamSpec(String id, String physicalName, String systemName, boolean isBounded, Map<String, String> config) {
+    this(id, physicalName, systemName, DEFAULT_PARTITION_COUNT, isBounded, config);
   }
 
   /**
@@ -140,9 +150,11 @@ public class StreamSpec {
    *
    * @param partitionCount  The number of partitionts for the stream. A value of {@code 1} indicates unpartitioned.
    *
+   * @param isBounded       The stream is bounded or not.
+   *
    * @param config          A map of properties for the stream. These may be System-specfic.
    */
-  public StreamSpec(String id, String physicalName, String systemName, int partitionCount,  Map<String, String> config) {
+  public StreamSpec(String id, String physicalName, String systemName, int partitionCount, boolean isBounded, Map<String, String> config) {
     validateLogicalIdentifier("streamId", id);
     validateLogicalIdentifier("systemName", systemName);
 
@@ -154,6 +166,7 @@ public class StreamSpec {
     this.systemName = systemName;
     this.physicalName = physicalName;
     this.partitionCount = partitionCount;
+    this.isBounded = isBounded;
 
     if (config != null) {
       this.config = Collections.unmodifiableMap(new HashMap<>(config));
@@ -171,7 +184,11 @@ public class StreamSpec {
    * @return                A copy of this StreamSpec with the specified partitionCount.
    */
   public StreamSpec copyWithPartitionCount(int partitionCount) {
-    return new StreamSpec(id, physicalName, systemName, partitionCount, config);
+    return new StreamSpec(id, physicalName, systemName, partitionCount, this.isBounded, config);
+  }
+
+  public StreamSpec copyWithPhysicalName(String physicalName) {
+    return new StreamSpec(id, physicalName, systemName, partitionCount, this.isBounded, config);
   }
 
   public String getId() {
@@ -214,6 +231,10 @@ public class StreamSpec {
     return id.equals(COORDINATOR_STREAM_ID);
   }
 
+  public boolean isBounded() {
+    return isBounded;
+  }
+
   private void validateLogicalIdentifier(String identifierName, String identifierValue) {
     if (identifierValue == null || !identifierValue.matches("[A-Za-z0-9_-]+")) {
       throw new IllegalArgumentException(String.format("Identifier '%s' is '%s'. It must match the expression [A-Za-z0-9_-]+", identifierName, identifierValue));
@@ -242,4 +263,8 @@ public class StreamSpec {
   public static StreamSpec createCoordinatorStreamSpec(String physicalName, String systemName) {
     return new StreamSpec(COORDINATOR_STREAM_ID, physicalName, systemName, 1);
   }
+
+  public static StreamSpec createCheckpointStreamSpec(String physicalName, String systemName) {
+    return new StreamSpec(CHECKPOINT_STREAM_ID, physicalName, systemName, 1);
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-core/src/main/java/org/apache/samza/config/ApplicationConfig.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/config/ApplicationConfig.java b/samza-core/src/main/java/org/apache/samza/config/ApplicationConfig.java
index 1b53321..39facb6 100644
--- a/samza-core/src/main/java/org/apache/samza/config/ApplicationConfig.java
+++ b/samza-core/src/main/java/org/apache/samza/config/ApplicationConfig.java
@@ -40,6 +40,11 @@ public class ApplicationConfig extends MapConfig {
   @Deprecated
   public static final String PROCESSOR_ID = "processor.id";
 
+  public enum ApplicationMode {
+    STREAM,
+    BATCH
+  }
+
   /**
    * Class implementing the {@link org.apache.samza.runtime.ProcessorIdGenerator} interface
    * Used to generate a unique identifier for a {@link org.apache.samza.processor.StreamProcessor} based on the runtime
@@ -49,6 +54,8 @@ public class ApplicationConfig extends MapConfig {
   public static final String APP_NAME = "app.name";
   public static final String APP_ID = "app.id";
   public static final String APP_CLASS = "app.class";
+  public static final String APP_MODE = "app.mode";
+  public static final String APP_RUN_ID = "app.run.id";
 
   public ApplicationConfig(Config config) {
     super(config);
@@ -83,4 +90,12 @@ public class ApplicationConfig extends MapConfig {
     return get(PROCESSOR_ID, null);
   }
 
+  public String getRunId() {
+    return get(APP_RUN_ID, null);
+  }
+
+  public ApplicationMode getAppMode() {
+    return ApplicationMode.valueOf(get(APP_MODE, ApplicationMode.STREAM.name()).toUpperCase());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-core/src/main/java/org/apache/samza/config/JavaStorageConfig.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/config/JavaStorageConfig.java b/samza-core/src/main/java/org/apache/samza/config/JavaStorageConfig.java
index c26601c..4e9a58a 100644
--- a/samza-core/src/main/java/org/apache/samza/config/JavaStorageConfig.java
+++ b/samza-core/src/main/java/org/apache/samza/config/JavaStorageConfig.java
@@ -22,6 +22,7 @@ package org.apache.samza.config;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.samza.SamzaException;
+import org.apache.samza.execution.StreamManager;
 
 
 /**
@@ -71,6 +72,8 @@ public class JavaStorageConfig extends MapConfig {
     } else {
       systemStreamRes = systemStream;
     }
+
+    systemStreamRes = StreamManager.createUniqueNameForBatch(systemStreamRes, this);
     return systemStreamRes;
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlan.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlan.java b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlan.java
index bde9bfb..3eb2e96 100644
--- a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlan.java
+++ b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlan.java
@@ -21,6 +21,7 @@ package org.apache.samza.execution;
 
 import java.util.List;
 import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.config.ApplicationConfig;
 import org.apache.samza.config.JobConfig;
 import org.apache.samza.system.StreamSpec;
 
@@ -39,6 +40,12 @@ public interface ExecutionPlan {
   List<JobConfig> getJobConfigs();
 
   /**
+   * Returns the config for this application
+   * @return {@link ApplicationConfig}
+   */
+  ApplicationConfig getApplicationConfig();
+
+  /**
    * Returns the intermediate streams that need to be created.
    * @return intermediate {@link StreamSpec}s
    */

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java
index 00f4ad4..e258d13 100644
--- a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java
+++ b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java
@@ -28,7 +28,10 @@ import java.util.LinkedList;
 import java.util.Map;
 import java.util.Queue;
 import java.util.Set;
+
 import org.apache.samza.SamzaException;
+import org.apache.samza.config.ApplicationConfig;
+import org.apache.samza.config.ClusterManagerConfig;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JobConfig;
 import org.apache.samza.operators.StreamGraphImpl;
@@ -56,6 +59,8 @@ public class ExecutionPlanner {
   }
 
   public ExecutionPlan plan(StreamGraphImpl streamGraph) throws Exception {
+    validateConfig();
+
     // create physical job graph based on stream graph
     JobGraph jobGraph = createJobGraph(streamGraph);
 
@@ -70,6 +75,16 @@ public class ExecutionPlanner {
     return jobGraph;
   }
 
+  private void validateConfig() {
+    ApplicationConfig appConfig = new ApplicationConfig(config);
+    ClusterManagerConfig clusterConfig = new ClusterManagerConfig(config);
+    // currently we don't support host-affinity in batch mode
+    if (appConfig.getAppMode() == ApplicationConfig.ApplicationMode.BATCH
+        && clusterConfig.getHostAffinityEnabled()) {
+      throw new SamzaException("Host affinity is not supported in batch mode. Please configure job.host-affinity.enabled=false.");
+    }
+  }
+
   /**
    * Create the physical graph from StreamGraph
    */

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java
index 99ee86c..2a09e90 100644
--- a/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java
+++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraph.java
@@ -173,7 +173,7 @@ import org.slf4j.LoggerFactory;
     String streamId = streamSpec.getId();
     StreamEdge edge = edges.get(streamId);
     if (edge == null) {
-      edge = new StreamEdge(streamSpec, isIntermediate);
+      edge = new StreamEdge(streamSpec, isIntermediate, config);
       edges.put(streamId, edge);
     }
     return edge;

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-core/src/main/java/org/apache/samza/execution/JobNode.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobNode.java b/samza-core/src/main/java/org/apache/samza/execution/JobNode.java
index 88b24ba..a86e019 100644
--- a/samza-core/src/main/java/org/apache/samza/execution/JobNode.java
+++ b/samza-core/src/main/java/org/apache/samza/execution/JobNode.java
@@ -29,14 +29,12 @@ import java.util.stream.Collectors;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JobConfig;
 import org.apache.samza.config.MapConfig;
-import org.apache.samza.config.StreamConfig;
 import org.apache.samza.config.TaskConfig;
 import org.apache.samza.operators.StreamGraphImpl;
 import org.apache.samza.operators.spec.JoinOperatorSpec;
 import org.apache.samza.operators.spec.OperatorSpec;
 import org.apache.samza.operators.spec.WindowOperatorSpec;
 import org.apache.samza.operators.util.MathUtils;
-import org.apache.samza.system.StreamSpec;
 import org.apache.samza.util.Util;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -125,7 +123,7 @@ public class JobNode {
     configs.put(CONFIG_INTERNAL_EXECUTION_PLAN, executionPlanJson);
 
     // write input/output streams to configs
-    inEdges.stream().filter(StreamEdge::isIntermediate).forEach(edge -> addStreamConfig(edge, configs));
+    inEdges.stream().filter(StreamEdge::isIntermediate).forEach(edge -> configs.putAll(edge.generateConfig()));
 
     log.info("Job {} has generated configs {}", jobName, configs);
 
@@ -190,18 +188,6 @@ public class JobNode {
     return scopedConfig;
   }
 
-  private static void addStreamConfig(StreamEdge edge, Map<String, String> config) {
-    StreamSpec spec = edge.getStreamSpec();
-    config.put(String.format(StreamConfig.SYSTEM_FOR_STREAM_ID(), spec.getId()), spec.getSystemName());
-    config.put(String.format(StreamConfig.PHYSICAL_NAME_FOR_STREAM_ID(), spec.getId()), spec.getPhysicalName());
-    if (edge.isIntermediate()) {
-      config.put(String.format(StreamConfig.IS_INTERMEDIATE_FROM_STREAM_ID(), spec.getId()), "true");
-    }
-    spec.getConfig().forEach((property, value) -> {
-        config.put(String.format(StreamConfig.STREAM_ID_PREFIX(), spec.getId()) + property, value);
-      });
-  }
-
   static String createId(String jobName, String jobId) {
     return String.format("%s-%s", jobName, jobId);
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-core/src/main/java/org/apache/samza/execution/StreamEdge.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/execution/StreamEdge.java b/samza-core/src/main/java/org/apache/samza/execution/StreamEdge.java
index 35fde81..f545490 100644
--- a/samza-core/src/main/java/org/apache/samza/execution/StreamEdge.java
+++ b/samza-core/src/main/java/org/apache/samza/execution/StreamEdge.java
@@ -20,7 +20,13 @@
 package org.apache.samza.execution;
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.config.StreamConfig;
 import org.apache.samza.system.StreamSpec;
 import org.apache.samza.system.SystemStream;
 import org.apache.samza.util.Util;
@@ -37,19 +43,21 @@ public class StreamEdge {
   private final StreamSpec streamSpec;
   private final List<JobNode> sourceNodes = new ArrayList<>();
   private final List<JobNode> targetNodes = new ArrayList<>();
+  private final Config config;
 
   private String name = "";
   private int partitions = PARTITIONS_UNKNOWN;
   private final boolean isIntermediate;
 
-  StreamEdge(StreamSpec streamSpec) {
-    this(streamSpec, false);
+  StreamEdge(StreamSpec streamSpec, Config config) {
+    this(streamSpec, false, config);
   }
 
-  StreamEdge(StreamSpec streamSpec, boolean isIntermediate) {
+  StreamEdge(StreamSpec streamSpec, boolean isIntermediate, Config config) {
     this.streamSpec = streamSpec;
     this.name = Util.getNameFromSystemStream(getSystemStream());
     this.isIntermediate = isIntermediate;
+    this.config = config;
   }
 
   void addSourceNode(JobNode sourceNode) {
@@ -60,12 +68,17 @@ public class StreamEdge {
     targetNodes.add(targetNode);
   }
 
-  public StreamSpec getStreamSpec() {
-    if (partitions == PARTITIONS_UNKNOWN) {
-      return streamSpec;
-    } else {
-      return streamSpec.copyWithPartitionCount(partitions);
+  StreamSpec getStreamSpec() {
+    StreamSpec spec = (partitions == PARTITIONS_UNKNOWN) ?
+        streamSpec : streamSpec.copyWithPartitionCount(partitions);
+
+    if (isIntermediate) {
+      String physicalName = StreamManager.createUniqueNameForBatch(spec.getPhysicalName(), config);
+      if (!physicalName.equals(spec.getPhysicalName())) {
+        spec = spec.copyWithPhysicalName(physicalName);
+      }
     }
+    return spec;
   }
 
   SystemStream getSystemStream() {
@@ -103,4 +116,22 @@ public class StreamEdge {
   boolean isIntermediate() {
     return isIntermediate;
   }
+
+  Config generateConfig() {
+    Map<String, String> config = new HashMap<>();
+    StreamSpec spec = getStreamSpec();
+    config.put(String.format(StreamConfig.SYSTEM_FOR_STREAM_ID(), spec.getId()), spec.getSystemName());
+    config.put(String.format(StreamConfig.PHYSICAL_NAME_FOR_STREAM_ID(), spec.getId()), spec.getPhysicalName());
+    if (isIntermediate()) {
+      config.put(String.format(StreamConfig.IS_INTERMEDIATE_FOR_STREAM_ID(), spec.getId()), "true");
+      config.put(String.format(StreamConfig.CONSUMER_OFFSET_DEFAULT_FOR_STREAM_ID(), spec.getId()), "oldest");
+    }
+    if (spec.isBounded()) {
+      config.put(String.format(StreamConfig.IS_BOUNDED_FOR_STREAM_ID(), spec.getId()), "true");
+    }
+    spec.getConfig().forEach((property, value) -> {
+        config.put(String.format(StreamConfig.STREAM_ID_PREFIX(), spec.getId()) + property, value);
+      });
+    return new MapConfig(config);
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-core/src/main/java/org/apache/samza/execution/StreamManager.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/execution/StreamManager.java b/samza-core/src/main/java/org/apache/samza/execution/StreamManager.java
index c6ab036..3028e5f 100644
--- a/samza-core/src/main/java/org/apache/samza/execution/StreamManager.java
+++ b/samza-core/src/main/java/org/apache/samza/execution/StreamManager.java
@@ -25,13 +25,23 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.stream.Collectors;
+
 import org.apache.samza.SamzaException;
+import org.apache.samza.checkpoint.CheckpointManager;
+import org.apache.samza.checkpoint.CheckpointManagerFactory;
+import org.apache.samza.config.*;
+import org.apache.samza.metrics.MetricsRegistryMap;
 import org.apache.samza.system.StreamSpec;
 import org.apache.samza.system.SystemAdmin;
+import org.apache.samza.system.SystemStream;
 import org.apache.samza.system.SystemStreamMetadata;
+import org.apache.samza.util.Util;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import scala.collection.JavaConversions;
 
+import static org.apache.samza.util.ScalaToJavaUtils.defaultValue;
 
 public class StreamManager {
   private static final Logger LOGGER = LoggerFactory.getLogger(StreamManager.class);
@@ -75,4 +85,70 @@ public class StreamManager {
 
     return streamToPartitionCount;
   }
+
+  /**
+   * This is a best-effort approach to clear the internal streams from previous run, including intermediate streams,
+   * checkpoint stream and changelog streams.
+   * For batch processing, we always clean up the previous internal streams and create a new set for each run.
+   * @param prevConfig config of the previous run
+   */
+  public void clearStreamsFromPreviousRun(Config prevConfig) {
+    try {
+      ApplicationConfig appConfig = new ApplicationConfig(prevConfig);
+      LOGGER.info("run.id from previous run is {}", appConfig.getRunId());
+
+      StreamConfig streamConfig = new StreamConfig(prevConfig);
+
+      //Find all intermediate streams and clean up
+      Set<StreamSpec> intStreams = JavaConversions.asJavaCollection(streamConfig.getStreamIds()).stream()
+          .filter(streamConfig::getIsIntermediate)
+          .map(id -> new StreamSpec(id, streamConfig.getPhysicalName(id), streamConfig.getSystem(id)))
+          .collect(Collectors.toSet());
+      intStreams.forEach(stream -> {
+          LOGGER.info("Clear intermediate stream {} in system {}", stream.getPhysicalName(), stream.getSystemName());
+          sysAdmins.get(stream.getSystemName()).clearStream(stream);
+        });
+
+      //Find checkpoint stream and clean up
+      TaskConfig taskConfig = new TaskConfig(prevConfig);
+      String checkpointManagerFactoryClass = taskConfig.getCheckpointManagerFactory().getOrElse(defaultValue(null));
+      if (checkpointManagerFactoryClass != null) {
+        CheckpointManager checkpointManager = ((CheckpointManagerFactory) Util.getObj(checkpointManagerFactoryClass))
+            .getCheckpointManager(prevConfig, new MetricsRegistryMap());
+        checkpointManager.clearCheckpoints();
+      }
+
+      //Find changelog streams and remove them
+      StorageConfig storageConfig = new StorageConfig(prevConfig);
+      for (String store : JavaConversions.asJavaCollection(storageConfig.getStoreNames())) {
+        String changelog = storageConfig.getChangelogStream(store).getOrElse(defaultValue(null));
+        if (changelog != null) {
+          LOGGER.info("Clear store {} changelog {}", store, changelog);
+          SystemStream systemStream = Util.getSystemStreamFromNames(changelog);
+          StreamSpec spec = StreamSpec.createChangeLogStreamSpec(systemStream.getStream(), systemStream.getSystem(), 1);
+          sysAdmins.get(spec.getSystemName()).clearStream(spec);
+        }
+      }
+    } catch (Exception e) {
+      // For batch, we always create a new set of internal streams (checkpoint, changelog and intermediate) with unique
+      // id. So if clearStream doesn't work, it won't affect the correctness of the results.
+      // We log a warning here and rely on retention to clean up the streams later.
+      LOGGER.warn("Fail to clear internal streams from previous run. Please clean up manually.", e);
+    }
+  }
+
+  /**
+   * Create a unique stream name if it's batch mode and has a valid run.id.
+   * @param stream physical name of the stream
+   * @param config {@link Config} object
+   * @return stream name created
+   */
+  public static String createUniqueNameForBatch(String stream, Config config) {
+    ApplicationConfig appConfig = new ApplicationConfig(config);
+    if (appConfig.getAppMode() == ApplicationConfig.ApplicationMode.BATCH && appConfig.getRunId() != null) {
+      return stream + "-" + appConfig.getRunId();
+    } else {
+      return stream;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java
index 3c7c83d..b8a8ca1 100644
--- a/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java
+++ b/samza-core/src/main/java/org/apache/samza/runtime/AbstractApplicationRunner.java
@@ -18,12 +18,13 @@
  */
 package org.apache.samza.runtime;
 
-import java.io.File;
-import java.io.PrintWriter;
-import java.util.Map;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.samza.application.StreamApplication;
+import org.apache.samza.config.ApplicationConfig;
+import org.apache.samza.config.ApplicationConfig.ApplicationMode;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JavaSystemConfig;
+import org.apache.samza.config.MapConfig;
 import org.apache.samza.config.ShellCommandConfig;
 import org.apache.samza.config.StreamConfig;
 import org.apache.samza.execution.ExecutionPlan;
@@ -34,6 +35,13 @@ import org.apache.samza.system.StreamSpec;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
+import java.io.PrintWriter;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
 
 /**
  * Defines common, core behavior for implementations of the {@link ApplicationRunner} API
@@ -42,12 +50,10 @@ public abstract class AbstractApplicationRunner extends ApplicationRunner {
   private static final Logger log = LoggerFactory.getLogger(AbstractApplicationRunner.class);
 
   private final StreamManager streamManager;
-  private final ExecutionPlanner planner;
 
   public AbstractApplicationRunner(Config config) {
     super(config);
     this.streamManager = new StreamManager(new JavaSystemConfig(config).getSystemAdmins());
-    this.planner = new ExecutionPlanner(config, streamManager);
   }
 
   @Override
@@ -96,20 +102,40 @@ public abstract class AbstractApplicationRunner extends ApplicationRunner {
   /*package private*/ StreamSpec getStreamSpec(String streamId, String physicalName, String system) {
     StreamConfig streamConfig = new StreamConfig(config);
     Map<String, String> properties = streamConfig.getStreamProperties(streamId);
+    boolean isBounded = streamConfig.getIsBounded(streamId);
+
+    return new StreamSpec(streamId, physicalName, system, isBounded, properties);
+  }
 
-    return new StreamSpec(streamId, physicalName, system, properties);
+  /* package private */
+  ExecutionPlan getExecutionPlan(StreamApplication app) throws Exception {
+    return getExecutionPlan(app, null);
   }
 
-  final ExecutionPlan getExecutionPlan(StreamApplication app) throws Exception {
+  /* package private */
+  ExecutionPlan getExecutionPlan(StreamApplication app, String runId) throws Exception {
     // build stream graph
     StreamGraphImpl streamGraph = new StreamGraphImpl(this, config);
     app.init(streamGraph, config);
 
     // create the physical execution plan
+    Map<String, String> cfg = new HashMap<>(config);
+    if (StringUtils.isNoneEmpty(runId)) {
+      cfg.put(ApplicationConfig.APP_RUN_ID, runId);
+    }
+
+    Set<StreamSpec> inputStreams = new HashSet<>(streamGraph.getInputOperators().keySet());
+    inputStreams.removeAll(streamGraph.getOutputStreams().keySet());
+    ApplicationMode mode = inputStreams.stream().allMatch(StreamSpec::isBounded)
+        ? ApplicationMode.BATCH : ApplicationMode.STREAM;
+    cfg.put(ApplicationConfig.APP_MODE, mode.name());
+
+    ExecutionPlanner planner = new ExecutionPlanner(new MapConfig(cfg), streamManager);
     return planner.plan(streamGraph);
   }
 
-  final StreamManager getStreamManager() {
+  /* package private for testing */
+  StreamManager getStreamManager() {
     return streamManager;
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java
index 53cd2f6..3e046af 100644
--- a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java
+++ b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java
@@ -21,11 +21,15 @@ package org.apache.samza.runtime;
 
 import org.apache.samza.SamzaException;
 import org.apache.samza.application.StreamApplication;
+import org.apache.samza.config.ApplicationConfig;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JobConfig;
+import org.apache.samza.coordinator.stream.CoordinatorStreamSystemConsumer;
+import org.apache.samza.coordinator.stream.CoordinatorStreamSystemFactory;
 import org.apache.samza.execution.ExecutionPlan;
 import org.apache.samza.job.ApplicationStatus;
 import org.apache.samza.job.JobRunner;
+import org.apache.samza.metrics.MetricsRegistryMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -53,11 +57,18 @@ public class RemoteApplicationRunner extends AbstractApplicationRunner {
   @Override
   public void run(StreamApplication app) {
     try {
+      // TODO: this is a tmp solution and the run.id generation will be addressed in another JIRA
+      String runId = String.valueOf(System.currentTimeMillis());
+      LOG.info("The run id for this run is {}", runId);
+
       // 1. initialize and plan
-      ExecutionPlan plan = getExecutionPlan(app);
+      ExecutionPlan plan = getExecutionPlan(app, runId);
       writePlanJsonFile(plan.getPlanAsJson());
 
       // 2. create the necessary streams
+      if (plan.getApplicationConfig().getAppMode() == ApplicationConfig.ApplicationMode.BATCH) {
+        getStreamManager().clearStreamsFromPreviousRun(getConfigFromPrevRun());
+      }
       getStreamManager().createStreams(plan.getIntermediateStreams());
 
       // 3. submit jobs for remote execution
@@ -133,4 +144,18 @@ public class RemoteApplicationRunner extends AbstractApplicationRunner {
       throw new SamzaException("Failed to get status for application", t);
     }
   }
+
+  private Config getConfigFromPrevRun() {
+    CoordinatorStreamSystemFactory coordinatorStreamSystemFactory = new CoordinatorStreamSystemFactory();
+    CoordinatorStreamSystemConsumer consumer = coordinatorStreamSystemFactory.getCoordinatorStreamSystemConsumer(
+        config, new MetricsRegistryMap());
+    consumer.register();
+    consumer.start();
+    consumer.bootstrap();
+    consumer.stop();
+
+    Config cfg = consumer.getConfig();
+    LOG.info("Previous config is: " + cfg.toString());
+    return cfg;
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinator.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinator.java b/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinator.java
index 2b8349c..4c4a645 100644
--- a/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinator.java
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinator.java
@@ -373,7 +373,6 @@ public class ZkJobCoordinator implements JobCoordinator, ZkControllerListener {
         throws Exception {
       LOG.info("Got new session created event for processor=" + processorId);
 
-
       LOG.info("register zk controller for the new session");
       zkController.register();
     }

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-core/src/main/scala/org/apache/samza/config/StreamConfig.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/config/StreamConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/StreamConfig.scala
index 20192fb..dd4b120 100644
--- a/samza-core/src/main/scala/org/apache/samza/config/StreamConfig.scala
+++ b/samza-core/src/main/scala/org/apache/samza/config/StreamConfig.scala
@@ -37,6 +37,7 @@ object StreamConfig {
   val BOOTSTRAP =               SAMZA_PROPERTY + "bootstrap"
   val PRIORITY =                SAMZA_PROPERTY + "priority"
   val IS_INTERMEDIATE =         SAMZA_PROPERTY + "intermediate"
+  val IS_BOUNDED =              SAMZA_PROPERTY + "bounded"
 
   // We don't want any external dependencies on these patterns while both exist. Use getProperty to ensure proper values.
   private val STREAMS_PREFIX = "streams."
@@ -45,7 +46,9 @@ object StreamConfig {
   val STREAM_ID_PREFIX = STREAMS_PREFIX + "%s."
   val SYSTEM_FOR_STREAM_ID = STREAM_ID_PREFIX + SYSTEM
   val PHYSICAL_NAME_FOR_STREAM_ID = STREAM_ID_PREFIX + PHYSICAL_NAME
-  val IS_INTERMEDIATE_FROM_STREAM_ID = STREAM_ID_PREFIX + IS_INTERMEDIATE
+  val IS_INTERMEDIATE_FOR_STREAM_ID = STREAM_ID_PREFIX + IS_INTERMEDIATE
+  val IS_BOUNDED_FOR_STREAM_ID = STREAM_ID_PREFIX + IS_BOUNDED
+  val CONSUMER_OFFSET_DEFAULT_FOR_STREAM_ID = STREAM_ID_PREFIX + CONSUMER_OFFSET_DEFAULT
 
   implicit def Config2Stream(config: Config) = new StreamConfig(config)
 }
@@ -159,7 +162,11 @@ class StreamConfig(config: Config) extends ScalaMapConfig(config) with Logging {
    * @return          true if the stream is intermediate
    */
   def getIsIntermediate(streamId: String) = {
-    getBoolean(StreamConfig.IS_INTERMEDIATE_FROM_STREAM_ID format streamId, false)
+    getBoolean(StreamConfig.IS_INTERMEDIATE_FOR_STREAM_ID format streamId, false)
+  }
+
+  def getIsBounded(streamId: String) = {
+    getBoolean(StreamConfig.IS_BOUNDED_FOR_STREAM_ID format streamId, false)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala b/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala
index 42bedec..92c3663 100644
--- a/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala
+++ b/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala
@@ -78,7 +78,7 @@ object JobModelManager extends Logging {
    * and writes it to the coordinator stream.
    * d) Builds JobModelManager using the jobModel read from coordinator stream.
    * @param coordinatorSystemConfig A config object that contains job.name
-   *                                job.id, and all system.&lt;job-coordinator-system-name&gt;.*
+   *                                job.id, and all system.&lt;job-coordinator-system-name&gt;.*Ch
    *                                configuration. The method will use this config to read all configuration
    *                                from the coordinator stream, and instantiate a JobModelManager.
    */

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-core/src/test/java/org/apache/samza/execution/TestExecutionPlanner.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestExecutionPlanner.java b/samza-core/src/test/java/org/apache/samza/execution/TestExecutionPlanner.java
index c4fd8f7..bab7159 100644
--- a/samza-core/src/test/java/org/apache/samza/execution/TestExecutionPlanner.java
+++ b/samza-core/src/test/java/org/apache/samza/execution/TestExecutionPlanner.java
@@ -371,13 +371,13 @@ public class TestExecutionPlanner {
   @Test
   public void testMaxPartition() {
     Collection<StreamEdge> edges = new ArrayList<>();
-    StreamEdge edge = new StreamEdge(input1);
+    StreamEdge edge = new StreamEdge(input1, config);
     edge.setPartitionCount(2);
     edges.add(edge);
-    edge = new StreamEdge(input2);
+    edge = new StreamEdge(input2, config);
     edge.setPartitionCount(32);
     edges.add(edge);
-    edge = new StreamEdge(input3);
+    edge = new StreamEdge(input3, config);
     edge.setPartitionCount(16);
     edges.add(edge);
 

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-core/src/test/java/org/apache/samza/execution/TestStreamEdge.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestStreamEdge.java b/samza-core/src/test/java/org/apache/samza/execution/TestStreamEdge.java
new file mode 100644
index 0000000..0a225f5
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/execution/TestStreamEdge.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.execution;
+
+import org.apache.samza.config.ApplicationConfig;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.config.StreamConfig;
+import org.apache.samza.system.StreamSpec;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestStreamEdge {
+  StreamSpec spec = new StreamSpec("stream-1", "physical-stream-1", "system-1");
+
+  @Test
+  public void testGetStreamSpec() {
+    StreamEdge edge = new StreamEdge(spec, false, new MapConfig());
+    assertEquals(edge.getStreamSpec(), spec);
+    assertEquals(edge.getStreamSpec().getPartitionCount(), 1 /*StreamSpec.DEFAULT_PARTITION_COUNT*/);
+
+    edge.setPartitionCount(10);
+    assertEquals(edge.getStreamSpec().getPartitionCount(), 10);
+  }
+
+  @Test
+  public void testGetStreamSpec_Batch() {
+    Map<String, String> config = new HashMap<>();
+    config.put(ApplicationConfig.APP_MODE, ApplicationConfig.ApplicationMode.BATCH.name());
+    config.put(ApplicationConfig.APP_RUN_ID, "123");
+    StreamEdge edge = new StreamEdge(spec, true, new MapConfig(config));
+    assertEquals(edge.getStreamSpec().getPhysicalName(), spec.getPhysicalName() + "-123");
+  }
+
+  @Test
+  public void testGenerateConfig() {
+    // an example unbounded IO stream
+    StreamSpec spec = new StreamSpec("stream-1", "physical-stream-1", "system-1", false, Collections.singletonMap("property1", "haha"));
+    StreamEdge edge = new StreamEdge(spec, false, new MapConfig());
+    Config config = edge.generateConfig();
+    StreamConfig streamConfig = new StreamConfig(config);
+    assertEquals(streamConfig.getSystem(spec.getId()), "system-1");
+    assertEquals(streamConfig.getPhysicalName(spec.getId()), "physical-stream-1");
+    assertEquals(streamConfig.getIsIntermediate(spec.getId()), false);
+    assertEquals(streamConfig.getIsBounded(spec.getId()), false);
+    assertEquals(streamConfig.getStreamProperties(spec.getId()).get("property1"), "haha");
+
+    // bounded stream
+    spec = new StreamSpec("stream-1", "physical-stream-1", "system-1", true, Collections.singletonMap("property1", "haha"));
+    edge = new StreamEdge(spec, false, new MapConfig());
+    config = edge.generateConfig();
+    streamConfig = new StreamConfig(config);
+    assertEquals(streamConfig.getIsBounded(spec.getId()), true);
+
+    // intermediate stream
+    edge = new StreamEdge(spec, true, new MapConfig());
+    config = edge.generateConfig();
+    streamConfig = new StreamConfig(config);
+    assertEquals(streamConfig.getIsIntermediate(spec.getId()), true);
+    assertEquals(streamConfig.getDefaultStreamOffset(spec.toSystemStream()).get(), "oldest");
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-core/src/test/java/org/apache/samza/execution/TestStreamManager.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestStreamManager.java b/samza-core/src/test/java/org/apache/samza/execution/TestStreamManager.java
new file mode 100644
index 0000000..dc36df8
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/execution/TestStreamManager.java
@@ -0,0 +1,147 @@
+/*
+ * 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.execution;
+
+import org.apache.samza.Partition;
+import org.apache.samza.checkpoint.CheckpointManager;
+import org.apache.samza.checkpoint.CheckpointManagerFactory;
+import org.apache.samza.config.ApplicationConfig;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.metrics.MetricsRegistry;
+import org.apache.samza.system.StreamSpec;
+import org.apache.samza.system.SystemAdmin;
+import org.apache.samza.system.SystemStreamMetadata;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+import java.util.*;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.*;
+
+public class TestStreamManager {
+  private static final String SYSTEM1 = "system-1";
+  private static final String SYSTEM2 = "system-2";
+  private static final String STREAM1 = "stream-1";
+  private static final String STREAM2 = "stream-2";
+
+  @Test
+  public void testCreateStreams() {
+    StreamSpec spec1 = new StreamSpec(STREAM1, STREAM1, SYSTEM1);
+    StreamSpec spec2 = new StreamSpec(STREAM2, STREAM2, SYSTEM2);
+    List<StreamSpec> specList = new ArrayList<>();
+    specList.add(spec1);
+    specList.add(spec2);
+
+    SystemAdmin admin1 = mock(SystemAdmin.class);
+    SystemAdmin admin2 = mock(SystemAdmin.class);
+    Map<String, SystemAdmin> sysAdmins = new HashMap<>();
+    sysAdmins.put(SYSTEM1, admin1);
+    sysAdmins.put(SYSTEM2, admin2);
+
+    StreamManager manager = new StreamManager(sysAdmins);
+    manager.createStreams(specList);
+
+    ArgumentCaptor<StreamSpec> captor = ArgumentCaptor.forClass(StreamSpec.class);
+    verify(admin1).createStream(captor.capture());
+    assertEquals(STREAM1, captor.getValue().getPhysicalName());
+
+    captor = ArgumentCaptor.forClass(StreamSpec.class);
+    verify(admin2).createStream(captor.capture());
+    assertEquals(STREAM2, captor.getValue().getPhysicalName());
+  }
+
+  @Test
+  public void testGetStreamPartitionCounts() {
+    SystemAdmin admin1 = mock(SystemAdmin.class);
+    Map<String, SystemAdmin> sysAdmins = new HashMap<>();
+    sysAdmins.put(SYSTEM1, admin1);
+
+    Map<String, SystemStreamMetadata> map = new HashMap<>();
+    SystemStreamMetadata meta1 = mock(SystemStreamMetadata.class);
+    Map<Partition, SystemStreamMetadata.SystemStreamPartitionMetadata> partitions = new HashMap<>();
+    partitions.put(new Partition(0), null);
+    when(meta1.getSystemStreamPartitionMetadata()).thenReturn(partitions);
+    map.put(STREAM1, meta1);
+
+    SystemStreamMetadata meta2 = mock(SystemStreamMetadata.class);
+    Map<Partition, SystemStreamMetadata.SystemStreamPartitionMetadata> partitions2 = new HashMap<>();
+    partitions2.put(new Partition(0), null);
+    partitions2.put(new Partition(1), null);
+    when(meta2.getSystemStreamPartitionMetadata()).thenReturn(partitions2);
+    map.put(STREAM2, meta2);
+
+    when(admin1.getSystemStreamMetadata(anyObject())).thenReturn(map);
+
+    Set<String> streams = new HashSet<>();
+    streams.add(STREAM1);
+    streams.add(STREAM2);
+    StreamManager manager = new StreamManager(sysAdmins);
+    Map<String, Integer> counts = manager.getStreamPartitionCounts(SYSTEM1, streams);
+
+    assertTrue(counts.get(STREAM1).equals(1));
+    assertTrue(counts.get(STREAM2).equals(2));
+  }
+
+  private static CheckpointManager checkpointManager = mock(CheckpointManager.class);
+  public static final class MockCheckpointManagerFactory implements CheckpointManagerFactory {
+    @Override
+    public CheckpointManager getCheckpointManager(Config config, MetricsRegistry registry) {
+      return checkpointManager;
+    }
+  }
+
+  @Test
+  public void testClearStreamsFromPreviousRun() {
+    SystemAdmin admin1 = mock(SystemAdmin.class);
+    SystemAdmin admin2 = mock(SystemAdmin.class);
+    Map<String, SystemAdmin> sysAdmins = new HashMap<>();
+    sysAdmins.put(SYSTEM1, admin1);
+    sysAdmins.put(SYSTEM2, admin2);
+
+    String runId = "123";
+    Map<String, String> config = new HashMap<>();
+    config.put(ApplicationConfig.APP_RUN_ID, "123");
+    config.put(ApplicationConfig.APP_MODE, ApplicationConfig.ApplicationMode.BATCH.name());
+
+    config.put("streams.stream-1.samza.system", SYSTEM1);
+    config.put("streams.stream-1.samza.physical.name", STREAM1 + "-" + runId);
+    config.put("streams.stream-1.samza.intermediate", "true");
+
+    config.put("task.checkpoint.factory", MockCheckpointManagerFactory.class.getName());
+    config.put("stores.test-store.factory", "dummyfactory");
+    config.put("stores.test-store.changelog", SYSTEM2 + "." + STREAM2);
+
+    StreamManager manager = new StreamManager(sysAdmins);
+    manager.clearStreamsFromPreviousRun(new MapConfig(config));
+
+    ArgumentCaptor<StreamSpec> captor = ArgumentCaptor.forClass(StreamSpec.class);
+    verify(admin1).clearStream(captor.capture());
+    assertEquals(captor.getValue().getPhysicalName(), STREAM1 + "-" + runId);
+
+    captor = ArgumentCaptor.forClass(StreamSpec.class);
+    verify(admin2).clearStream(captor.capture());
+    assertEquals(captor.getValue().getPhysicalName(), STREAM2 + "-" + runId);
+
+    verify(checkpointManager, times(1)).clearCheckpoints();
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
index 5b2c661..d2094b4 100644
--- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
+++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
@@ -20,13 +20,9 @@
 package org.apache.samza.runtime;
 
 import com.google.common.collect.ImmutableList;
-import java.lang.reflect.Field;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 import java.util.stream.Collectors;
-import java.util.Set;
+
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.ApplicationConfig;
 import org.apache.samza.config.JobConfig;
@@ -36,7 +32,6 @@ import org.apache.samza.coordinator.CoordinationUtils;
 import org.apache.samza.coordinator.CoordinationUtilsFactory;
 import org.apache.samza.coordinator.DistributedLockWithState;
 import org.apache.samza.execution.ExecutionPlan;
-import org.apache.samza.execution.ExecutionPlanner;
 import org.apache.samza.execution.StreamManager;
 import org.apache.samza.job.ApplicationStatus;
 import org.apache.samza.processor.StreamProcessor;
@@ -54,6 +49,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.mockito.Matchers.anyObject;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.*;
+import static org.powermock.api.mockito.PowerMockito.doReturn;
 import static org.powermock.api.mockito.PowerMockito.mockStatic;
 
 
@@ -72,51 +68,29 @@ public class TestLocalApplicationRunner {
   public void testStreamCreation()
       throws Exception {
     Map<String, String> config = new HashMap<>();
-    LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(config));
+    LocalApplicationRunner runner = spy(new LocalApplicationRunner(new MapConfig(config)));
     StreamApplication app = mock(StreamApplication.class);
     doNothing().when(app).init(anyObject(), anyObject());
 
-    ExecutionPlanner planner = mock(ExecutionPlanner.class);
-    Field plannerField = runner.getClass().getSuperclass().getDeclaredField("planner");
-    plannerField.setAccessible(true);
-    plannerField.set(runner, planner);
-
     StreamManager streamManager = mock(StreamManager.class);
-    Field streamManagerField = runner.getClass().getSuperclass().getDeclaredField("streamManager");
-    streamManagerField.setAccessible(true);
-    streamManagerField.set(runner, streamManager);
-    ArgumentCaptor<List> captor = ArgumentCaptor.forClass(List.class);
+    doReturn(streamManager).when(runner).getStreamManager();
 
-    ExecutionPlan plan = new ExecutionPlan() {
-      @Override
-      public List<JobConfig> getJobConfigs() {
-        return Collections.emptyList();
-      }
-
-      @Override
-      public List<StreamSpec> getIntermediateStreams() {
-        return Collections.singletonList(new StreamSpec("test-stream", "test-stream", "test-system"));
-      }
-
-      @Override
-      public String getPlanAsJson()
-          throws Exception {
-        return "";
-      }
-    };
-    when(planner.plan(anyObject())).thenReturn(plan);
+    ExecutionPlan plan = mock(ExecutionPlan.class);
+    when(plan.getIntermediateStreams()).thenReturn(Collections.singletonList(new StreamSpec("test-stream", "test-stream", "test-system")));
+    when(plan.getPlanAsJson()).thenReturn("");
+    doReturn(plan).when(runner).getExecutionPlan(any(), any());
 
     mockStatic(CoordinationUtilsFactory.class);
     CoordinationUtilsFactory coordinationUtilsFactory = mock(CoordinationUtilsFactory.class);
     when(CoordinationUtilsFactory.getCoordinationUtilsFactory(anyObject())).thenReturn(coordinationUtilsFactory);
 
-    LocalApplicationRunner spy = spy(runner);
     try {
-      spy.run(app);
+      runner.run(app);
     } catch (Throwable t) {
       assertNotNull(t); //no jobs exception
     }
 
+    ArgumentCaptor<List> captor = ArgumentCaptor.forClass(List.class);
     verify(streamManager).createStreams(captor.capture());
     List<StreamSpec> streamSpecs = captor.getValue();
     assertEquals(streamSpecs.size(), 1);
@@ -127,41 +101,19 @@ public class TestLocalApplicationRunner {
   public void testStreamCreationWithCoordination()
       throws Exception {
     Map<String, String> config = new HashMap<>();
-    LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(config));
+    LocalApplicationRunner localRunner = new LocalApplicationRunner(new MapConfig(config));
+    LocalApplicationRunner runner = spy(localRunner);
+
     StreamApplication app = mock(StreamApplication.class);
     doNothing().when(app).init(anyObject(), anyObject());
 
-    ExecutionPlanner planner = mock(ExecutionPlanner.class);
-    Field plannerField = runner.getClass().getSuperclass().getDeclaredField("planner");
-    plannerField.setAccessible(true);
-    plannerField.set(runner, planner);
-
     StreamManager streamManager = mock(StreamManager.class);
-    Field streamManagerField = runner.getClass().getSuperclass().getDeclaredField("streamManager");
-    streamManagerField.setAccessible(true);
-    streamManagerField.set(runner, streamManager);
-    ArgumentCaptor<List> captor = ArgumentCaptor.forClass(List.class);
+    doReturn(streamManager).when(runner).getStreamManager();
 
-    ExecutionPlan plan = new ExecutionPlan() {
-      @Override
-      public List<JobConfig> getJobConfigs() {
-        return Collections.emptyList();
-      }
-
-      @Override
-      public List<StreamSpec> getIntermediateStreams() {
-        return Collections.singletonList(new StreamSpec("test-stream", "test-stream", "test-system"));
-      }
-
-      @Override
-      public String getPlanAsJson()
-          throws Exception {
-        return "";
-      }
-    };
-    when(planner.plan(anyObject())).thenReturn(plan);
-
-    LocalApplicationRunner spy = spy(runner);
+    ExecutionPlan plan = mock(ExecutionPlan.class);
+    when(plan.getIntermediateStreams()).thenReturn(Collections.singletonList(new StreamSpec("test-stream", "test-stream", "test-system")));
+    when(plan.getPlanAsJson()).thenReturn("");
+    doReturn(plan).when(runner).getExecutionPlan(any(), any());
 
     CoordinationUtils coordinationUtils = mock(CoordinationUtils.class);
     CoordinationUtilsFactory coordinationUtilsFactory = mock(CoordinationUtilsFactory.class);
@@ -175,12 +127,14 @@ public class TestLocalApplicationRunner {
         .thenReturn(coordinationUtils);
 
     try {
-      spy.run(app);
+      runner.run(app);
     } catch (Throwable t) {
       assertNotNull(t); //no jobs exception
     }
 
+    ArgumentCaptor<List> captor = ArgumentCaptor.forClass(List.class);
     verify(streamManager).createStreams(captor.capture());
+
     List<StreamSpec> streamSpecs = captor.getValue();
     assertEquals(streamSpecs.size(), 1);
     assertEquals(streamSpecs.get(0).getId(), "test-stream");
@@ -220,33 +174,15 @@ public class TestLocalApplicationRunner {
       throws Exception {
     final Map<String, String> config = new HashMap<>();
     config.put(ApplicationConfig.APP_PROCESSOR_ID_GENERATOR_CLASS, UUIDGenerator.class.getName());
-    LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(config));
+    LocalApplicationRunner runner = spy(new LocalApplicationRunner(new MapConfig(config)));
     StreamApplication app = mock(StreamApplication.class);
     doNothing().when(app).init(anyObject(), anyObject());
 
-    ExecutionPlanner planner = mock(ExecutionPlanner.class);
-    Field plannerField = runner.getClass().getSuperclass().getDeclaredField("planner");
-    plannerField.setAccessible(true);
-    plannerField.set(runner, planner);
-
-    ExecutionPlan plan = new ExecutionPlan() {
-      @Override
-      public List<JobConfig> getJobConfigs() {
-        return Collections.singletonList(new JobConfig(new MapConfig(config)));
-      }
-
-      @Override
-      public List<StreamSpec> getIntermediateStreams() {
-        return Collections.emptyList();
-      }
-
-      @Override
-      public String getPlanAsJson()
-          throws Exception {
-        return "";
-      }
-    };
-    when(planner.plan(anyObject())).thenReturn(plan);
+    ExecutionPlan plan = mock(ExecutionPlan.class);
+    when(plan.getIntermediateStreams()).thenReturn(Collections.emptyList());
+    when(plan.getPlanAsJson()).thenReturn("");
+    when(plan.getJobConfigs()).thenReturn(Collections.singletonList(new JobConfig(new MapConfig(config))));
+    doReturn(plan).when(runner).getExecutionPlan(any(), any());
 
     StreamProcessor sp = mock(StreamProcessor.class);
     ArgumentCaptor<StreamProcessorLifecycleListener> captor =
@@ -260,12 +196,11 @@ public class TestLocalApplicationRunner {
         return null;
       }).when(sp).start();
 
-    LocalApplicationRunner spy = spy(runner);
-    doReturn(sp).when(spy).createStreamProcessor(anyObject(), anyObject(), captor.capture());
+    doReturn(sp).when(runner).createStreamProcessor(anyObject(), anyObject(), captor.capture());
 
-    spy.run(app);
+    runner.run(app);
 
-    assertEquals(spy.status(app), ApplicationStatus.SuccessfulFinish);
+    assertEquals(runner.status(app), ApplicationStatus.SuccessfulFinish);
   }
 
   @Test
@@ -273,33 +208,15 @@ public class TestLocalApplicationRunner {
       throws Exception {
     final Map<String, String> config = new HashMap<>();
     config.put(ApplicationConfig.PROCESSOR_ID, "0");
-    LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(config));
+    LocalApplicationRunner runner = spy(new LocalApplicationRunner(new MapConfig(config)));
     StreamApplication app = mock(StreamApplication.class);
     doNothing().when(app).init(anyObject(), anyObject());
 
-    ExecutionPlanner planner = mock(ExecutionPlanner.class);
-    Field plannerField = runner.getClass().getSuperclass().getDeclaredField("planner");
-    plannerField.setAccessible(true);
-    plannerField.set(runner, planner);
-
-    ExecutionPlan plan = new ExecutionPlan() {
-      @Override
-      public List<JobConfig> getJobConfigs() {
-        return Collections.singletonList(new JobConfig(new MapConfig(config)));
-      }
-
-      @Override
-      public List<StreamSpec> getIntermediateStreams() {
-        return Collections.emptyList();
-      }
-
-      @Override
-      public String getPlanAsJson()
-          throws Exception {
-        return "";
-      }
-    };
-    when(planner.plan(anyObject())).thenReturn(plan);
+    ExecutionPlan plan = mock(ExecutionPlan.class);
+    when(plan.getIntermediateStreams()).thenReturn(Collections.emptyList());
+    when(plan.getPlanAsJson()).thenReturn("");
+    when(plan.getJobConfigs()).thenReturn(Collections.singletonList(new JobConfig(new MapConfig(config))));
+    doReturn(plan).when(runner).getExecutionPlan(any(), any());
 
     Throwable t = new Throwable("test failure");
     StreamProcessor sp = mock(StreamProcessor.class);
@@ -313,16 +230,15 @@ public class TestLocalApplicationRunner {
         return null;
       }).when(sp).start();
 
-    LocalApplicationRunner spy = spy(runner);
-    doReturn(sp).when(spy).createStreamProcessor(anyObject(), anyObject(), captor.capture());
+    doReturn(sp).when(runner).createStreamProcessor(anyObject(), anyObject(), captor.capture());
 
     try {
-      spy.run(app);
+      runner.run(app);
     } catch (Throwable th) {
       assertNotNull(th);
     }
 
-    assertEquals(spy.status(app), ApplicationStatus.UnsuccessfulFinish);
+    assertEquals(runner.status(app), ApplicationStatus.UnsuccessfulFinish);
   }
 
   public static Set<StreamProcessor> getProcessors(LocalApplicationRunner runner) {

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaStreamSpec.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaStreamSpec.java b/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaStreamSpec.java
index c7e82f7..fd53a45 100644
--- a/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaStreamSpec.java
+++ b/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaStreamSpec.java
@@ -149,7 +149,7 @@ public class KafkaStreamSpec extends StreamSpec {
    */
   public KafkaStreamSpec(String id, String topicName, String systemName, int partitionCount, int replicationFactor,
       Properties properties) {
-    super(id, topicName, systemName, partitionCount, propertiesToMap(properties));
+    super(id, topicName, systemName, partitionCount, false, propertiesToMap(properties));
 
     if (replicationFactor <= 0) {
       throw new IllegalArgumentException(

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala b/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala
index 6461f9d..c8b7a9b 100644
--- a/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala
+++ b/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala
@@ -33,8 +33,10 @@ import org.apache.kafka.common.utils.Utils
 import org.apache.kafka.clients.producer.{Producer, ProducerRecord}
 import org.apache.samza.SamzaException
 import org.apache.samza.checkpoint.{Checkpoint, CheckpointManager}
+import org.apache.samza.config.JavaSystemConfig
 import org.apache.samza.container.TaskName
 import org.apache.samza.serializers.CheckpointSerde
+import org.apache.samza.system.{StreamSpec, SystemAdmin}
 import org.apache.samza.system.kafka.TopicMetadataCache
 import org.apache.samza.util._
 
@@ -62,7 +64,8 @@ class KafkaCheckpointManager(
                               failOnCheckpointValidation: Boolean,
                               val retryBackoff: ExponentialSleepStrategy = new ExponentialSleepStrategy,
                               serde: CheckpointSerde = new CheckpointSerde,
-                              checkpointTopicProperties: Properties = new Properties) extends CheckpointManager with Logging {
+                              checkpointTopicProperties: Properties = new Properties,
+                              systemAdmin: SystemAdmin = null) extends CheckpointManager with Logging {
   import org.apache.samza.checkpoint.kafka.KafkaCheckpointManager._
 
   var taskNames = Set[TaskName]()
@@ -72,6 +75,8 @@ class KafkaCheckpointManager(
   var startingOffset: Option[Long] = None // Where to start reading for each subsequent call of readCheckpoint
   val kafkaUtil: KafkaUtil = new KafkaUtil(retryBackoff, connectZk)
 
+
+
   KafkaCheckpointLogKey.setSystemStreamPartitionGrouperFactoryString(systemStreamPartitionGrouperFactoryString)
 
   info("Creating KafkaCheckpointManager with: clientId=%s, checkpointTopic=%s, systemName=%s" format(clientId, checkpointTopic, systemName))
@@ -275,22 +280,28 @@ class KafkaCheckpointManager(
 
   }
 
-  def start {
+  override def start {
     kafkaUtil.createTopic(checkpointTopic, 1, replicationFactor, checkpointTopicProperties)
     kafkaUtil.validateTopicPartitionCount(checkpointTopic, systemName, metadataStore, 1, failOnCheckpointValidation)
   }
 
-  def register(taskName: TaskName) {
+  override def register(taskName: TaskName) {
     debug("Adding taskName " + taskName + " to " + this)
     taskNames += taskName
   }
 
-  def stop = {
+  override def stop = {
     if (producer != null) {
       producer.close
     }
   }
 
+  override def clearCheckpoints = {
+    info("Clear checkpoint stream %s in system %s" format (checkpointTopic, systemName))
+    val spec = StreamSpec.createCheckpointStreamSpec(checkpointTopic, systemName)
+    systemAdmin.clearStream(spec)
+  }
+
   override def toString = "KafkaCheckpointManager [systemName=%s, checkpointTopic=%s]" format(systemName, checkpointTopic)
 }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManagerFactory.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManagerFactory.scala b/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManagerFactory.scala
index c42882e..0df581f 100644
--- a/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManagerFactory.scala
+++ b/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManagerFactory.scala
@@ -27,9 +27,10 @@ import org.apache.samza.SamzaException
 import org.apache.samza.checkpoint.{CheckpointManager, CheckpointManagerFactory}
 import org.apache.samza.config.JobConfig.Config2Job
 import org.apache.samza.config.KafkaConfig.Config2Kafka
-import org.apache.samza.config.{Config, KafkaConfig}
+import org.apache.samza.config.{SystemConfig, JavaSystemConfig, Config, KafkaConfig}
 import org.apache.samza.metrics.MetricsRegistry
-import org.apache.samza.util.{ClientUtilTopicMetadataStore, KafkaUtil, Logging}
+import org.apache.samza.system.{SystemFactory, SystemAdmin}
+import org.apache.samza.util.{Util, ClientUtilTopicMetadataStore, KafkaUtil, Logging}
 
 object KafkaCheckpointManagerFactory {
   val INJECTED_PRODUCER_PROPERTIES = Map(
@@ -82,10 +83,14 @@ class KafkaCheckpointManagerFactory extends CheckpointManagerFactory with Loggin
     }
     val socketTimeout = consumerConfig.socketTimeoutMs
 
+    val systemConfig = new SystemConfig(config)
+    val systemFactoryClassName = systemConfig.getSystemFactory(systemName).get
+    val systemFactory: SystemFactory = Util.getObj(systemFactoryClassName)
+    val systemAdmin = systemFactory.getAdmin(systemName, config)
 
     new KafkaCheckpointManager(
       clientId,
-      KafkaUtil.getCheckpointTopic(jobName, jobId),
+      KafkaUtil.getCheckpointTopic(jobName, jobId, config),
       systemName,
       config.getCheckpointReplicationFactor.getOrElse("3").toInt,
       socketTimeout,
@@ -96,6 +101,7 @@ class KafkaCheckpointManagerFactory extends CheckpointManagerFactory with Loggin
       connectZk,
       config.getSystemStreamPartitionGrouperFactory,      // To find out the SSPGrouperFactory class so it can be included/verified in the key
       config.failOnCheckpointValidation,
-      checkpointTopicProperties = getCheckpointTopicProperties(config))
+      checkpointTopicProperties = getCheckpointTopicProperties(config),
+      systemAdmin = systemAdmin)
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala
index 6e582e9..a2256c8 100644
--- a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala
@@ -402,6 +402,7 @@ class KafkaSystemAdmin(
    * @inheritdoc
    */
   override def createStream(spec: StreamSpec): Boolean = {
+    info("Create topic %s in system %s" format (spec.getPhysicalName, systemName))
     val kSpec = toKafkaSpec(spec)
     var streamCreated = false
 
@@ -501,6 +502,7 @@ class KafkaSystemAdmin(
    * Otherwise it's a no-op.
    */
   override def clearStream(spec: StreamSpec): Boolean = {
+    info("Delete topic %s in system %s" format (spec.getPhysicalName, systemName))
     val kSpec = KafkaStreamSpec.fromSpec(spec)
     var retries = CLEAR_STREAM_RETRIES
     new ExponentialSleepStrategy().run(

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-kafka/src/main/scala/org/apache/samza/util/KafkaUtil.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/util/KafkaUtil.scala b/samza-kafka/src/main/scala/org/apache/samza/util/KafkaUtil.scala
index 41d380b..39edba7 100644
--- a/samza-kafka/src/main/scala/org/apache/samza/util/KafkaUtil.scala
+++ b/samza-kafka/src/main/scala/org/apache/samza/util/KafkaUtil.scala
@@ -24,9 +24,10 @@ import java.util.concurrent.atomic.AtomicLong
 import kafka.admin.AdminUtils
 import kafka.utils.ZkUtils
 import org.apache.kafka.common.PartitionInfo
-import org.apache.samza.config.Config
-import org.apache.samza.config.ConfigException
+import org.apache.samza.config.ApplicationConfig.ApplicationMode
+import org.apache.samza.config.{ApplicationConfig, Config, ConfigException}
 import org.apache.samza.config.JobConfig.Config2Job
+import org.apache.samza.execution.StreamManager
 import org.apache.samza.system.OutgoingMessageEnvelope
 import kafka.common.{ErrorMapping, ReplicaNotAvailableException}
 import org.apache.kafka.common.errors.TopicExistsException
@@ -57,8 +58,11 @@ object KafkaUtil extends Logging {
     abs(envelope.getPartitionKey.hashCode()) % numPartitions
   }
 
-  def getCheckpointTopic(jobName: String, jobId: String) =
-    "__samza_checkpoint_ver_%d_for_%s_%s" format (CHECKPOINT_LOG_VERSION_NUMBER, jobName.replaceAll("_", "-"), jobId.replaceAll("_", "-"))
+  def getCheckpointTopic(jobName: String, jobId: String, config: Config) = {
+    val checkpointTopic = "__samza_checkpoint_ver_%d_for_%s_%s" format(CHECKPOINT_LOG_VERSION_NUMBER,
+      jobName.replaceAll("_", "-"), jobId.replaceAll("_", "-"))
+    StreamManager.createUniqueNameForBatch(checkpointTopic, config)
+  }
 
   /**
    * Exactly the same as Kafka's ErrorMapping.maybeThrowException

http://git-wip-us.apache.org/repos/asf/samza/blob/47541488/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaStreamSpec.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaStreamSpec.java b/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaStreamSpec.java
index 69345a3..5612704 100644
--- a/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaStreamSpec.java
+++ b/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaStreamSpec.java
@@ -33,7 +33,7 @@ public class TestKafkaStreamSpec {
 
   @Test
   public void testUnsupportedConfigStrippedFromProperties() {
-    StreamSpec original = new StreamSpec("dummyId","dummyPhysicalName", "dummySystemName", ImmutableMap.of("segment.bytes", "4", "replication.factor", "7"));
+    StreamSpec original = new StreamSpec("dummyId","dummyPhysicalName", "dummySystemName", false, ImmutableMap.of("segment.bytes", "4", "replication.factor", "7"));
 
     // First verify the original
     assertEquals("7", original.get("replication.factor"));


[07/16] samza git commit: Merge branch 'master' into 0.14.0

Posted by xi...@apache.org.
Merge branch 'master' into 0.14.0


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

Branch: refs/heads/master
Commit: 1701ea84a2e029b3297687b5fc814998371b1a6f
Parents: 79200c7 fb39a51
Author: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>
Authored: Tue Sep 12 11:32:36 2017 -0700
Committer: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>
Committed: Tue Sep 12 11:32:36 2017 -0700

----------------------------------------------------------------------
 KEYS                                            |  57 +++
 NOTICE                                          |   1 +
 README.md                                       |   2 +-
 bin/check-all.sh                                |   2 +-
 build.gradle                                    |   4 +-
 docs/contribute/tests.md                        |   2 +-
 .../versioned/jobs/configuration-table.html     |  14 +-
 .../versioned/hello-samza-high-level-yarn.md    |   2 +-
 .../versioned/hello-samza-high-level-zk.md      |   2 +-
 .../versioned/samza-rest-getting-started.md     |   2 +-
 docs/startup/download/index.md                  |  17 +-
 docs/startup/hello-samza/versioned/index.md     |   2 +-
 docs/startup/preview/index.md                   |   2 +-
 .../main/java/org/apache/samza/AzureClient.java |  20 +-
 .../main/java/org/apache/samza/AzureConfig.java |  73 ---
 .../main/java/org/apache/samza/BlobUtils.java   | 280 ----------
 .../java/org/apache/samza/JobModelBundle.java   |  61 ---
 .../java/org/apache/samza/LeaseBlobManager.java |  98 ----
 .../java/org/apache/samza/ProcessorEntity.java  |  58 ---
 .../main/java/org/apache/samza/TableUtils.java  | 198 --------
 .../org/apache/samza/config/AzureConfig.java    |  68 +++
 .../coordinator/AzureCoordinationUtils.java     |  58 +++
 .../AzureCoordinationUtilsFactory.java          |  30 ++
 .../samza/coordinator/AzureJobCoordinator.java  | 509 +++++++++++++++++++
 .../coordinator/AzureJobCoordinatorFactory.java |  29 ++
 .../samza/coordinator/AzureLeaderElector.java   | 109 ++++
 .../org/apache/samza/coordinator/AzureLock.java | 100 ++++
 .../samza/coordinator/DistributedLock.java      |  39 ++
 .../samza/coordinator/data/BarrierState.java    |  27 +
 .../samza/coordinator/data/JobModelBundle.java  |  61 +++
 .../samza/coordinator/data/ProcessorEntity.java |  62 +++
 .../scheduler/HeartbeatScheduler.java           |  81 +++
 .../scheduler/JMVersionUpgradeScheduler.java    |  99 ++++
 .../LeaderBarrierCompleteScheduler.java         | 118 +++++
 .../scheduler/LeaderLivenessCheckScheduler.java | 120 +++++
 .../scheduler/LivenessCheckScheduler.java       | 108 ++++
 .../scheduler/RenewLeaseScheduler.java          |  79 +++
 .../scheduler/SchedulerStateChangeListener.java |  29 ++
 .../coordinator/scheduler/TaskScheduler.java    |  35 ++
 .../java/org/apache/samza/util/BlobUtils.java   | 284 +++++++++++
 .../org/apache/samza/util/LeaseBlobManager.java |  99 ++++
 .../java/org/apache/samza/util/TableUtils.java  | 205 ++++++++
 .../samza/config/JobCoordinatorConfig.java      |  21 +
 .../apache/samza/container/LocalityManager.java |   6 +-
 .../coordinator/CoordinationServiceFactory.java |  36 --
 .../samza/coordinator/CoordinationUtils.java    |  14 +-
 .../coordinator/CoordinationUtilsFactory.java   |  47 ++
 .../coordinator/DistributedLockWithState.java   |  42 ++
 .../samza/runtime/LocalApplicationRunner.java   |  91 ++--
 .../org/apache/samza/task/AsyncRunLoop.java     |  14 +
 .../samza/zk/ZkBarrierForVersionUpgrade.java    |   2 +-
 .../org/apache/samza/zk/ZkControllerImpl.java   |  19 +-
 .../samza/zk/ZkCoordinationServiceFactory.java  |  89 ----
 .../apache/samza/zk/ZkCoordinationUtils.java    |  26 +-
 .../samza/zk/ZkCoordinationUtilsFactory.java    |  89 ++++
 .../org/apache/samza/zk/ZkDistributedLock.java  | 117 +++++
 .../samza/zk/ZkJobCoordinatorFactory.java       |  23 +-
 .../org/apache/samza/zk/ZkLeaderElector.java    |   2 +-
 .../org/apache/samza/zk/ZkProcessorLatch.java   |  23 +-
 .../main/java/org/apache/samza/zk/ZkUtils.java  |  41 +-
 .../apache/samza/checkpoint/OffsetManager.scala |  50 +-
 .../apache/samza/container/TaskInstance.scala   |   4 +-
 .../runtime/TestApplicationRunnerMain.java      |   2 +-
 .../runtime/TestLocalApplicationRunner.java     | 185 ++++---
 .../org/apache/samza/task/TestAsyncRunLoop.java |  24 +-
 .../apache/samza/zk/TestZkLeaderElector.java    |   7 +-
 .../org/apache/samza/zk/TestZkNamespace.java    |   8 +-
 .../apache/samza/zk/TestZkProcessorLatch.java   |   2 +-
 .../java/org/apache/samza/zk/TestZkUtils.java   |  57 ++-
 .../samza/checkpoint/TestOffsetManager.scala    |  58 ++-
 .../samza/config/TestJobCoordinatorConfig.java  |  58 +++
 .../samza/container/TestTaskInstance.scala      |  62 ++-
 .../org/apache/samza/rest/SamzaRestService.java |  14 +-
 .../processor/TestZkStreamProcessorSession.java |   3 +-
 .../processor/TestZkLocalApplicationRunner.java |  60 ++-
 .../test/integration/TestStatefulTask.scala     |  79 ++-
 settings.gradle                                 |  17 +
 77 files changed, 3444 insertions(+), 1194 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/1701ea84/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/1701ea84/samza-core/src/main/java/org/apache/samza/task/AsyncRunLoop.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/1701ea84/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/1701ea84/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
----------------------------------------------------------------------
diff --cc samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
index 4be4e73,f9c1252..5b2c661
--- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
+++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
@@@ -19,7 -19,13 +19,14 @@@
  
  package org.apache.samza.runtime;
  
+ import com.google.common.collect.ImmutableList;
+ import java.lang.reflect.Field;
+ import java.util.Collections;
+ import java.util.HashMap;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.stream.Collectors;
 +import java.util.Set;
  import org.apache.samza.application.StreamApplication;
  import org.apache.samza.config.ApplicationConfig;
  import org.apache.samza.config.JobConfig;
@@@ -343,8 -324,73 +325,77 @@@ public class TestLocalApplicationRunne
      assertEquals(spy.status(app), ApplicationStatus.UnsuccessfulFinish);
    }
  
 +  public static Set<StreamProcessor> getProcessors(LocalApplicationRunner runner) {
 +    return runner.getProcessors();
 +  }
 +
+   /**
+    * A test case to verify if the plan results in different hash if there is change in topological sort order.
+    * Note: the overall JOB PLAN remains the same outside the scope of intermediate streams the sake of these test cases.
+    */
+   @Test
+   public void testPlanIdWithShuffledStreamSpecs() {
+     List<StreamSpec> streamSpecs = ImmutableList.of(new StreamSpec("test-stream-1", "stream-1", "testStream"),
+         new StreamSpec("test-stream-2", "stream-2", "testStream"),
+         new StreamSpec("test-stream-3", "stream-3", "testStream"));
+     String planIdBeforeShuffle = getExecutionPlanId(streamSpecs);
+ 
+     List<StreamSpec> shuffledStreamSpecs = ImmutableList.of(new StreamSpec("test-stream-2", "stream-2", "testStream"),
+         new StreamSpec("test-stream-1", "stream-1", "testStream"),
+         new StreamSpec("test-stream-3", "stream-3", "testStream"));
+ 
+ 
+     assertFalse("Expected both of the latch ids to be different",
+         planIdBeforeShuffle.equals(getExecutionPlanId(shuffledStreamSpecs)));
+   }
+ 
+   /**
+    * A test case to verify if the plan results in same hash in case of same plan.
+    * Note: the overall JOB PLAN remains the same outside the scope of intermediate streams the sake of these test cases.
+    */
+   @Test
+   public void testGeneratePlanIdWithSameStreamSpecs() {
+     List<StreamSpec> streamSpecs = ImmutableList.of(new StreamSpec("test-stream-1", "stream-1", "testStream"),
+         new StreamSpec("test-stream-2", "stream-2", "testStream"),
+         new StreamSpec("test-stream-3", "stream-3", "testStream"));
+     String planIdForFirstAttempt = getExecutionPlanId(streamSpecs);
+     String planIdForSecondAttempt = getExecutionPlanId(streamSpecs);
+ 
+     assertEquals("Expected latch ids to match!", "1447946713", planIdForFirstAttempt);
+     assertEquals("Expected latch ids to match for the second attempt!", planIdForFirstAttempt, planIdForSecondAttempt);
+   }
+ 
+   /**
+    * A test case to verify plan results in different hash in case of different intermediate stream.
+    * Note: the overall JOB PLAN remains the same outside the scope of intermediate streams the sake of these test cases.
+    */
+   @Test
+   public void testGeneratePlanIdWithDifferentStreamSpecs() {
+     List<StreamSpec> streamSpecs = ImmutableList.of(new StreamSpec("test-stream-1", "stream-1", "testStream"),
+         new StreamSpec("test-stream-2", "stream-2", "testStream"),
+         new StreamSpec("test-stream-3", "stream-3", "testStream"));
+     String planIdBeforeShuffle = getExecutionPlanId(streamSpecs);
+ 
+     List<StreamSpec> updatedStreamSpecs = ImmutableList.of(new StreamSpec("test-stream-1", "stream-1", "testStream"),
+         new StreamSpec("test-stream-4", "stream-4", "testStream"),
+         new StreamSpec("test-stream-3", "stream-3", "testStream"));
+ 
+ 
+     assertFalse("Expected both of the latch ids to be different",
+         planIdBeforeShuffle.equals(getExecutionPlanId(updatedStreamSpecs)));
+   }
+ 
+   private String getExecutionPlanId(List<StreamSpec> updatedStreamSpecs) {
+     String intermediateStreamJson =
+         updatedStreamSpecs.stream().map(this::streamSpecToJson).collect(Collectors.joining(","));
+ 
+     int planId = String.format(PLAN_JSON, intermediateStreamJson).hashCode();
+ 
+     return String.valueOf(planId);
+   }
+ 
+   private String streamSpecToJson(StreamSpec streamSpec) {
+     return String.format(STREAM_SPEC_JSON_FORMAT, streamSpec.getId(), streamSpec.getId(), streamSpec.getSystemName(),
+         streamSpec.getPhysicalName());
+   }
  }

http://git-wip-us.apache.org/repos/asf/samza/blob/1701ea84/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/1701ea84/samza-core/src/test/scala/org/apache/samza/checkpoint/TestOffsetManager.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/1701ea84/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
----------------------------------------------------------------------
diff --cc samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
index dcb06d3,4958a57..81f3ed1
--- a/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
+++ b/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
@@@ -19,34 -19,16 +19,32 @@@
  
  package org.apache.samza.container
  
 +
 +import java.util
 +import java.util
 +import java.util.Collections
  import java.util.concurrent.ConcurrentHashMap
 +import com.google.common.collect.Multimap
 +import org.apache.samza.SamzaException
+ 
  import org.apache.samza.Partition
 +import org.apache.samza.checkpoint.OffsetManager
 +import org.apache.samza.config.Config
 +import org.apache.samza.config.MapConfig
 +import org.apache.samza.control.ControlMessageUtils
 +import org.apache.samza.job.model.ContainerModel
 +import org.apache.samza.job.model.JobModel
 +import org.apache.samza.job.model.TaskModel
 +import org.apache.samza.metrics.Counter
 +import org.apache.samza.metrics.Metric
 +import org.apache.samza.metrics.MetricsRegistryMap
+ import org.apache.samza.checkpoint.{Checkpoint, OffsetManager}
+ import org.apache.samza.config.{Config, MapConfig}
+ import org.apache.samza.metrics.{Counter, Metric, MetricsRegistryMap}
  import org.apache.samza.serializers.SerdeManager
- import org.apache.samza.system.IncomingMessageEnvelope
- import org.apache.samza.system.SystemConsumer
- import org.apache.samza.system.SystemConsumers
- import org.apache.samza.system.SystemProducer
- import org.apache.samza.system.SystemProducers
- import org.apache.samza.system.SystemStream
- import org.apache.samza.system.SystemStreamMetadata
+ import org.apache.samza.storage.TaskStorageManager
  import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata
- import org.apache.samza.system.SystemStreamPartition
+ import org.apache.samza.system._
  import org.apache.samza.system.chooser.RoundRobinChooser
  import org.apache.samza.task._
  import org.junit.Assert._
@@@ -365,34 -350,47 +366,77 @@@ class TestTaskInstance 
    }
  
    @Test
+   def testCommitOrder {
+     // Simple objects
+     val partition = new Partition(0)
+     val taskName = new TaskName("taskName")
+     val systemStream = new SystemStream("test-system", "test-stream")
+     val systemStreamPartition = new SystemStreamPartition(systemStream, partition)
+     val checkpoint = new Checkpoint(Map(systemStreamPartition -> "4").asJava)
+ 
+     // Mocks
+     val collector = Mockito.mock(classOf[TaskInstanceCollector])
+     val storageManager = Mockito.mock(classOf[TaskStorageManager])
+     val offsetManager = Mockito.mock(classOf[OffsetManager])
+     when(offsetManager.buildCheckpoint(any())).thenReturn(checkpoint)
+     val mockOrder = inOrder(offsetManager, collector, storageManager)
+ 
+     val taskInstance: TaskInstance = new TaskInstance(
+       Mockito.mock(classOf[StreamTask]).asInstanceOf[StreamTask],
+       taskName,
+       new MapConfig,
+       new TaskInstanceMetrics,
+       null,
+       Mockito.mock(classOf[SystemConsumers]),
+       collector,
+       Mockito.mock(classOf[SamzaContainerContext]),
+       offsetManager,
+       storageManager,
+       systemStreamPartitions = Set(systemStreamPartition))
+ 
+     taskInstance.commit
+ 
+     // We must first get a snapshot of the checkpoint so it doesn't change while we flush. SAMZA-1384
+     mockOrder.verify(offsetManager).buildCheckpoint(taskName)
+     // Producers must be flushed next and ideally the output would be flushed before the changelog
+     // s.t. the changelog and checkpoints (state and inputs) are captured last
+     mockOrder.verify(collector).flush
+     // Local state is next, to ensure that the state (particularly the offset file) never points to a newer changelog
+     // offset than what is reflected in the on disk state.
+     mockOrder.verify(storageManager).flush()
+     // Finally, checkpoint the inputs with the snapshotted checkpoint captured at the beginning of commit
+     mockOrder.verify(offsetManager).writeCheckpoint(taskName, checkpoint)
+   }
++
++  @Test
 +  def testBuildInputToTasks = {
 +    val system: String = "test-system"
 +    val stream0: String = "test-stream-0"
 +    val stream1: String = "test-stream-1"
 +
 +    val ssp0: SystemStreamPartition = new SystemStreamPartition(system, stream0, new Partition(0))
 +    val ssp1: SystemStreamPartition = new SystemStreamPartition(system, stream0, new Partition(1))
 +    val ssp2: SystemStreamPartition = new SystemStreamPartition(system, stream1, new Partition(0))
 +
 +    val task0: TaskName = new TaskName("Task 0")
 +    val task1: TaskName = new TaskName("Task 1")
 +    val ssps: util.Set[SystemStreamPartition] = new util.HashSet[SystemStreamPartition]
 +    ssps.add(ssp0)
 +    ssps.add(ssp2)
 +    val tm0: TaskModel = new TaskModel(task0, ssps, new Partition(0))
 +    val cm0: ContainerModel = new ContainerModel("c0", 0, Collections.singletonMap(task0, tm0))
 +    val tm1: TaskModel = new TaskModel(task1, Collections.singleton(ssp1), new Partition(1))
 +    val cm1: ContainerModel = new ContainerModel("c1", 1, Collections.singletonMap(task1, tm1))
 +
 +    val cms: util.Map[String, ContainerModel] = new util.HashMap[String, ContainerModel]
 +    cms.put(cm0.getProcessorId, cm0)
 +    cms.put(cm1.getProcessorId, cm1)
 +
 +    val jobModel: JobModel = new JobModel(new MapConfig, cms, null)
 +    val streamToTasks: Multimap[SystemStream, String] = TaskInstance.buildInputToTasks(jobModel)
 +    assertEquals(streamToTasks.get(ssp0.getSystemStream).size, 2)
 +    assertEquals(streamToTasks.get(ssp2.getSystemStream).size, 1)
 +  }
  }
  
  class MockSystemAdmin extends SystemAdmin {


[16/16] samza git commit: Merge branch '0.14.0'

Posted by xi...@apache.org.
Merge branch '0.14.0'


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

Branch: refs/heads/master
Commit: db2f121e22ad2ae28f914dd1510d251e713d2e7c
Parents: 6e6e53b 052a057
Author: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>
Authored: Tue Oct 3 15:10:13 2017 -0700
Committer: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>
Committed: Tue Oct 3 15:10:13 2017 -0700

----------------------------------------------------------------------
 .../versioned/jobs/configuration-table.html     |  32 +++
 .../samza/checkpoint/CheckpointManager.java     |   4 +
 .../operators/functions/WatermarkFunction.java  |  58 ++++++
 .../org/apache/samza/system/ControlMessage.java |  46 +++++
 .../apache/samza/system/EndOfStreamMessage.java |  38 ++++
 .../samza/system/IncomingMessageEnvelope.java   |   9 +-
 .../org/apache/samza/system/MessageType.java    |  45 ++++
 .../org/apache/samza/system/StreamSpec.java     |  68 +++++-
 .../org/apache/samza/system/SystemAdmin.java    |  42 +---
 .../apache/samza/system/WatermarkMessage.java   |  46 +++++
 ...inglePartitionWithoutOffsetsSystemAdmin.java |  16 --
 .../apache/samza/config/ApplicationConfig.java  |  15 ++
 .../apache/samza/config/JavaStorageConfig.java  |   5 +
 .../apache/samza/container/TaskContextImpl.java | 131 ++++++++++++
 .../apache/samza/execution/ExecutionPlan.java   |   7 +
 .../samza/execution/ExecutionPlanner.java       |  20 +-
 .../org/apache/samza/execution/JobGraph.java    |   2 +-
 .../org/apache/samza/execution/JobNode.java     |  14 +-
 .../org/apache/samza/execution/StreamEdge.java  |  50 ++++-
 .../apache/samza/execution/StreamManager.java   |  76 +++++++
 .../apache/samza/message/ControlMessage.java    |  52 -----
 .../samza/message/EndOfStreamMessage.java       |  36 ----
 .../org/apache/samza/message/MessageType.java   |  46 -----
 .../apache/samza/message/WatermarkMessage.java  |  43 ----
 .../operators/impl/ControlMessageSender.java    |  56 +++++
 .../samza/operators/impl/EndOfStreamStates.java |  98 +++++++++
 .../samza/operators/impl/OperatorImpl.java      | 191 ++++++++++++++++-
 .../samza/operators/impl/OperatorImplGraph.java |  96 ++++++++-
 .../operators/impl/PartitionByOperatorImpl.java |  26 +++
 .../samza/operators/impl/WatermarkStates.java   | 119 +++++++++++
 .../samza/operators/spec/InputOperatorSpec.java |   6 +
 .../samza/operators/spec/JoinOperatorSpec.java  |   6 +
 .../samza/operators/spec/OperatorSpec.java      |   3 +
 .../operators/spec/OutputOperatorSpec.java      |   7 +
 .../operators/spec/PartitionByOperatorSpec.java |   5 +
 .../samza/operators/spec/SinkOperatorSpec.java  |   6 +
 .../operators/spec/StreamOperatorSpec.java      |   6 +
 .../operators/spec/WindowOperatorSpec.java      |   8 +
 .../apache/samza/processor/StreamProcessor.java |  18 +-
 .../runtime/AbstractApplicationRunner.java      |  42 +++-
 .../samza/runtime/LocalApplicationRunner.java   |   5 +
 .../samza/runtime/LocalContainerRunner.java     |   6 +-
 .../samza/runtime/RemoteApplicationRunner.java  |  27 ++-
 .../serializers/IntermediateMessageSerde.java   |   6 +-
 .../PassthroughCoordinationUtilsFactory.java    |  30 +++
 .../org/apache/samza/task/AsyncRunLoop.java     |  12 +-
 .../apache/samza/task/StreamOperatorTask.java   |  27 ++-
 .../samza/zk/ScheduleAfterDebounceTime.java     | 172 +++++++++++-----
 .../org/apache/samza/zk/ZkJobCoordinator.java   |  27 ++-
 .../apache/samza/checkpoint/OffsetManager.scala |   7 +-
 .../org/apache/samza/config/StreamConfig.scala  |  11 +-
 .../apache/samza/container/SamzaContainer.scala |  13 +-
 .../apache/samza/container/TaskInstance.scala   |  88 ++++----
 .../samza/coordinator/JobModelManager.scala     |  10 +-
 .../scala/org/apache/samza/job/JobRunner.scala  |   9 +-
 .../samza/job/local/ThreadJobFactory.scala      |   6 +-
 .../apache/samza/serializers/SerdeManager.scala |   5 +-
 .../samza/storage/TaskStorageManager.scala      |  12 +-
 .../samza/system/StreamMetadataCache.scala      |  11 +
 .../MockCoordinatorStreamSystemFactory.java     |  13 +-
 .../samza/execution/TestExecutionPlanner.java   |  21 +-
 .../org/apache/samza/execution/TestJobNode.java |   7 +-
 .../apache/samza/execution/TestStreamEdge.java  |  84 ++++++++
 .../samza/execution/TestStreamManager.java      | 147 +++++++++++++
 .../samza/operators/TestJoinOperator.java       |   3 +-
 .../samza/operators/TestWindowOperator.java     |   6 +-
 .../impl/TestControlMessageSender.java          |  73 +++++++
 .../operators/impl/TestEndOfStreamStates.java   |  78 +++++++
 .../samza/operators/impl/TestOperatorImpl.java  |  46 ++++-
 .../operators/impl/TestOperatorImplGraph.java   | 187 +++++++++++++++--
 .../operators/impl/TestWatermarkStates.java     | 102 +++++++++
 .../samza/processor/TestStreamProcessor.java    |   4 +-
 .../runtime/TestLocalApplicationRunner.java     | 162 ++++-----------
 .../TestIntermediateMessageSerde.java           |  12 +-
 .../org/apache/samza/task/TestAsyncRunLoop.java |   2 +-
 .../samza/task/TestStreamOperatorTask.java      |  30 +++
 .../samza/zk/TestScheduleAfterDebounceTime.java |  74 ++++++-
 .../samza/checkpoint/TestOffsetManager.scala    |  12 --
 .../samza/container/TestTaskInstance.scala      |  22 +-
 .../samza/coordinator/TestJobCoordinator.scala  |  12 --
 .../samza/serializers/TestSerdeManager.scala    |  10 +-
 .../samza/storage/TestTaskStorageManager.scala  |   6 +-
 .../elasticsearch/ElasticsearchSystemAdmin.java |  15 --
 .../samza/system/hdfs/HdfsSystemAdmin.java      |  15 --
 .../samza/system/kafka/KafkaStreamSpec.java     |   2 +-
 .../kafka/KafkaCheckpointManager.scala          |  19 +-
 .../kafka/KafkaCheckpointManagerFactory.scala   |  14 +-
 .../samza/system/kafka/KafkaSystemAdmin.scala   |  96 +++++----
 .../scala/org/apache/samza/util/KafkaUtil.scala |  12 +-
 .../samza/system/kafka/TestKafkaStreamSpec.java |   2 +-
 .../system/kafka/TestKafkaSystemAdminJava.java  | 166 +++++++--------
 .../system/kafka/TestKafkaSystemAdmin.scala     |   5 +-
 .../samza/system/mock/MockSystemAdmin.java      |  15 --
 .../processor/TestStreamProcessorUtil.java      |  28 +++
 .../EndOfStreamIntegrationTest.java             | 112 ++++++++++
 .../samza/test/controlmessages/TestData.java    |  85 ++++++++
 .../WatermarkIntegrationTest.java               | 206 +++++++++++++++++++
 .../samza/test/util/ArraySystemConsumer.java    |  82 ++++++++
 .../samza/test/util/ArraySystemFactory.java     |  50 +++++
 .../samza/test/util/Base64Serializer.java       |  66 ++++++
 .../samza/test/util/SimpleSystemAdmin.java      |  75 +++++++
 .../samza/test/util/TestStreamConsumer.java     |  52 +++++
 .../apache/samza/job/yarn/MockSystemAdmin.scala |  12 --
 103 files changed, 3451 insertions(+), 869 deletions(-)
----------------------------------------------------------------------



[12/16] samza git commit: SAMZA-1386: Inline End-of-stream and Watermark logic inside OperatorImpl

Posted by xi...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java
index 99496eb..9b747bc 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java
@@ -18,9 +18,14 @@
  */
 package org.apache.samza.operators.impl;
 
+import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Multimap;
+import java.util.stream.Collectors;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.samza.config.Config;
+import org.apache.samza.container.TaskContextImpl;
+import org.apache.samza.job.model.JobModel;
 import org.apache.samza.operators.StreamGraphImpl;
 import org.apache.samza.operators.functions.JoinFunction;
 import org.apache.samza.operators.functions.PartialJoinFunction;
@@ -36,6 +41,8 @@ import org.apache.samza.storage.kv.KeyValueStore;
 import org.apache.samza.system.SystemStream;
 import org.apache.samza.task.TaskContext;
 import org.apache.samza.util.Clock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -50,6 +57,7 @@ import java.util.Map;
  * The DAG of {@link OperatorImpl}s corresponding to the DAG of {@link OperatorSpec}s.
  */
 public class OperatorImplGraph {
+  private static final Logger LOG = LoggerFactory.getLogger(OperatorImplGraph.class);
 
   /**
    * A mapping from operator names to their {@link OperatorImpl}s in this graph. Used to avoid creating
@@ -84,10 +92,27 @@ public class OperatorImplGraph {
    */
   public OperatorImplGraph(StreamGraphImpl streamGraph, Config config, TaskContext context, Clock clock) {
     this.clock = clock;
+
+    TaskContextImpl taskContext = (TaskContextImpl) context;
+    Map<SystemStream, Integer> producerTaskCounts = hasIntermediateStreams(streamGraph) ?
+        getProducerTaskCountForIntermediateStreams(getStreamToConsumerTasks(taskContext.getJobModel()),
+            getIntermediateToInputStreamsMap(streamGraph)) :
+        Collections.EMPTY_MAP;
+    producerTaskCounts.forEach((stream, count) -> {
+        LOG.info("{} has {} producer tasks.", stream, count);
+      });
+
+    // set states for end-of-stream
+    taskContext.registerObject(EndOfStreamStates.class.getName(),
+        new EndOfStreamStates(context.getSystemStreamPartitions(), producerTaskCounts));
+    // set states for watermark
+    taskContext.registerObject(WatermarkStates.class.getName(),
+        new WatermarkStates(context.getSystemStreamPartitions(), producerTaskCounts));
+
     streamGraph.getInputOperators().forEach((streamSpec, inputOpSpec) -> {
         SystemStream systemStream = new SystemStream(streamSpec.getSystemName(), streamSpec.getPhysicalName());
         InputOperatorImpl inputOperatorImpl =
-            (InputOperatorImpl) createAndRegisterOperatorImpl(null, inputOpSpec, config, context);
+            (InputOperatorImpl) createAndRegisterOperatorImpl(null, inputOpSpec, systemStream, config, context);
         this.inputOperators.put(systemStream, inputOperatorImpl);
       });
   }
@@ -128,17 +153,18 @@ public class OperatorImplGraph {
    * @return  the operator implementation for the operatorSpec
    */
   OperatorImpl createAndRegisterOperatorImpl(OperatorSpec prevOperatorSpec, OperatorSpec operatorSpec,
-      Config config, TaskContext context) {
+      SystemStream inputStream, Config config, TaskContext context) {
     if (!operatorImpls.containsKey(operatorSpec.getOpName()) || operatorSpec instanceof JoinOperatorSpec) {
       // Either this is the first time we've seen this operatorSpec, or this is a join operator spec
       // and we need to create 2 partial join operator impls for it. Initialize and register the sub-DAG.
       OperatorImpl operatorImpl = createOperatorImpl(prevOperatorSpec, operatorSpec, config, context);
       operatorImpl.init(config, context);
+      operatorImpl.registerInputStream(inputStream);
       operatorImpls.put(operatorImpl.getOperatorName(), operatorImpl);
 
       Collection<OperatorSpec> registeredSpecs = operatorSpec.getRegisteredOperatorSpecs();
       registeredSpecs.forEach(registeredSpec -> {
-          OperatorImpl nextImpl = createAndRegisterOperatorImpl(operatorSpec, registeredSpec, config, context);
+          OperatorImpl nextImpl = createAndRegisterOperatorImpl(operatorSpec, registeredSpec, inputStream, config, context);
           operatorImpl.registerNextOperator(nextImpl);
         });
       return operatorImpl;
@@ -246,4 +272,70 @@ public class OperatorImplGraph {
       }
     };
   }
+
+  private boolean hasIntermediateStreams(StreamGraphImpl streamGraph) {
+    return !Collections.disjoint(streamGraph.getInputOperators().keySet(), streamGraph.getOutputStreams().keySet());
+  }
+
+  /**
+   * calculate the task count that produces to each intermediate streams
+   * @param streamToConsumerTasks input streams to task mapping
+   * @param intermediateToInputStreams intermediate stream to input streams mapping
+   * @return mapping from intermediate stream to task count
+   */
+  static Map<SystemStream, Integer> getProducerTaskCountForIntermediateStreams(
+      Multimap<SystemStream, String> streamToConsumerTasks,
+      Multimap<SystemStream, SystemStream> intermediateToInputStreams) {
+    Map<SystemStream, Integer> result = new HashMap<>();
+    intermediateToInputStreams.asMap().entrySet().forEach(entry -> {
+        result.put(entry.getKey(),
+            entry.getValue().stream()
+                .flatMap(systemStream -> streamToConsumerTasks.get(systemStream).stream())
+                .collect(Collectors.toSet()).size());
+      });
+    return result;
+  }
+
+  /**
+   * calculate the mapping from input streams to consumer tasks
+   * @param jobModel JobModel object
+   * @return mapping from input stream to tasks
+   */
+  static Multimap<SystemStream, String> getStreamToConsumerTasks(JobModel jobModel) {
+    Multimap<SystemStream, String> streamToConsumerTasks = HashMultimap.create();
+    jobModel.getContainers().values().forEach(containerModel -> {
+        containerModel.getTasks().values().forEach(taskModel -> {
+            taskModel.getSystemStreamPartitions().forEach(ssp -> {
+                streamToConsumerTasks.put(ssp.getSystemStream(), taskModel.getTaskName().getTaskName());
+              });
+          });
+      });
+    return streamToConsumerTasks;
+  }
+
+  /**
+   * calculate the mapping from output streams to input streams
+   * @param streamGraph the user {@link StreamGraphImpl} instance
+   * @return mapping from output streams to input streams
+   */
+  static Multimap<SystemStream, SystemStream> getIntermediateToInputStreamsMap(StreamGraphImpl streamGraph) {
+    Multimap<SystemStream, SystemStream> outputToInputStreams = HashMultimap.create();
+    streamGraph.getInputOperators().entrySet().stream()
+        .forEach(
+            entry -> computeOutputToInput(entry.getKey().toSystemStream(), entry.getValue(), outputToInputStreams));
+    return outputToInputStreams;
+  }
+
+  private static void computeOutputToInput(SystemStream input, OperatorSpec opSpec,
+      Multimap<SystemStream, SystemStream> outputToInputStreams) {
+    if (opSpec instanceof OutputOperatorSpec) {
+      OutputOperatorSpec outputOpSpec = (OutputOperatorSpec) opSpec;
+      if (outputOpSpec.getOpCode() == OperatorSpec.OpCode.PARTITION_BY) {
+        outputToInputStreams.put(outputOpSpec.getOutputStream().getStreamSpec().toSystemStream(), input);
+      }
+    } else {
+      Collection<OperatorSpec> nextOperators = opSpec.getRegisteredOperatorSpecs();
+      nextOperators.forEach(spec -> computeOutputToInput(input, spec, outputToInputStreams));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/operators/impl/OutputOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/OutputOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/OutputOperatorImpl.java
index f212b3e..205bba6 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/impl/OutputOperatorImpl.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/OutputOperatorImpl.java
@@ -21,12 +21,16 @@ package org.apache.samza.operators.impl;
 import java.util.Collection;
 import java.util.Collections;
 import org.apache.samza.config.Config;
-import org.apache.samza.control.Watermark;
+import org.apache.samza.container.TaskContextImpl;
+import org.apache.samza.system.ControlMessage;
+import org.apache.samza.system.EndOfStreamMessage;
 import org.apache.samza.operators.spec.OperatorSpec;
 import org.apache.samza.operators.spec.OutputOperatorSpec;
 import org.apache.samza.operators.spec.OutputStreamImpl;
 import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.StreamMetadataCache;
 import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.WatermarkMessage;
 import org.apache.samza.task.MessageCollector;
 import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
@@ -39,10 +43,16 @@ class OutputOperatorImpl<M> extends OperatorImpl<M, Void> {
 
   private final OutputOperatorSpec<M> outputOpSpec;
   private final OutputStreamImpl<?, ?, M> outputStream;
+  private final String taskName;
+  private final ControlMessageSender controlMessageSender;
 
   OutputOperatorImpl(OutputOperatorSpec<M> outputOpSpec, Config config, TaskContext context) {
     this.outputOpSpec = outputOpSpec;
     this.outputStream = outputOpSpec.getOutputStream();
+    this.taskName = context.getTaskName().getTaskName();
+
+    StreamMetadataCache streamMetadataCache = ((TaskContextImpl) context).getStreamMetadataCache();
+    this.controlMessageSender = new ControlMessageSender(streamMetadataCache);
   }
 
   @Override
@@ -71,12 +81,22 @@ class OutputOperatorImpl<M> extends OperatorImpl<M, Void> {
   }
 
   @Override
-  protected long handleWatermark(Watermark inputWatermark,
-      MessageCollector collector,
-      TaskCoordinator coordinator) {
+  protected void handleEndOfStream(MessageCollector collector, TaskCoordinator coordinator) {
     if (outputOpSpec.getOpCode() == OperatorSpec.OpCode.PARTITION_BY) {
-      inputWatermark.propagate(outputStream.getStreamSpec().toSystemStream());
+      sendControlMessage(new EndOfStreamMessage(taskName), collector);
     }
-    return inputWatermark.getTimestamp();
+  }
+
+  @Override
+  protected Long handleWatermark(long watermark, MessageCollector collector, TaskCoordinator coordinator) {
+    if (outputOpSpec.getOpCode() == OperatorSpec.OpCode.PARTITION_BY) {
+      sendControlMessage(new WatermarkMessage(watermark, taskName), collector);
+    }
+    return watermark;
+  }
+
+  private void sendControlMessage(ControlMessage message, MessageCollector collector) {
+    SystemStream outputStream = outputOpSpec.getOutputStream().getStreamSpec().toSystemStream();
+    controlMessageSender.send(message, outputStream, collector);
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/operators/impl/WatermarkStates.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/WatermarkStates.java b/samza-core/src/main/java/org/apache/samza/operators/impl/WatermarkStates.java
new file mode 100644
index 0000000..0295626
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/WatermarkStates.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.impl;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.system.WatermarkMessage;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class manages the watermarks coming from input/intermediate streams in a task. Internally it keeps track
+ * of the latest watermark timestamp from each upstream task, and use the min as the consolidated watermark time.
+ *
+ * This class is thread-safe. However, having parallelism within a task may result in out-of-order processing
+ * and inaccurate watermarks. In this scenario, watermarks might be emitted before the previous messages fully processed.
+ */
+class WatermarkStates {
+  private static final Logger LOG = LoggerFactory.getLogger(WatermarkStates.class);
+
+  public static final long WATERMARK_NOT_EXIST = -1;
+
+  private final static class WatermarkState {
+    private final int expectedTotal;
+    private final Map<String, Long> timestamps = new HashMap<>();
+    private volatile long watermarkTime = WATERMARK_NOT_EXIST;
+
+    WatermarkState(int expectedTotal) {
+      this.expectedTotal = expectedTotal;
+    }
+
+    synchronized void update(long timestamp, String taskName) {
+      if (taskName != null) {
+        Long ts = timestamps.get(taskName);
+        if (ts != null && ts > timestamp) {
+          LOG.warn(String.format("Incoming watermark %s is smaller than existing watermark %s for upstream task %s",
+              timestamp, ts, taskName));
+        } else {
+          timestamps.put(taskName, timestamp);
+        }
+      }
+
+      /**
+       * Check whether we got all the watermarks.
+       * At a sources, the expectedTotal is 0.
+       * For any intermediate streams, the expectedTotal is the upstream task count.
+       */
+      if (timestamps.size() == expectedTotal) {
+        Optional<Long> min = timestamps.values().stream().min(Long::compare);
+        watermarkTime = min.orElse(timestamp);
+      }
+    }
+
+    long getWatermarkTime() {
+      return watermarkTime;
+    }
+  }
+
+  private final Map<SystemStreamPartition, WatermarkState> watermarkStates;
+
+  WatermarkStates(Set<SystemStreamPartition> ssps, Map<SystemStream, Integer> producerTaskCounts) {
+    Map<SystemStreamPartition, WatermarkState> states = new HashMap<>();
+    ssps.forEach(ssp -> {
+        states.put(ssp, new WatermarkState(producerTaskCounts.getOrDefault(ssp.getSystemStream(), 0)));
+      });
+    this.watermarkStates = Collections.unmodifiableMap(states);
+  }
+
+  /**
+   * Update the state upon receiving a watermark message.
+   * @param watermarkMessage message of {@link WatermarkMessage}
+   * @param ssp system stream partition
+   * @return true iff the stream has a new watermark
+   */
+  void update(WatermarkMessage watermarkMessage, SystemStreamPartition ssp) {
+    WatermarkState state = watermarkStates.get(ssp);
+    if (state != null) {
+      state.update(watermarkMessage.getTimestamp(), watermarkMessage.getTaskName());
+    } else {
+      LOG.error("SSP {} doesn't have watermark states", ssp);
+    }
+  }
+
+  long getWatermark(SystemStream systemStream) {
+    return watermarkStates.entrySet().stream()
+        .filter(entry -> entry.getKey().getSystemStream().equals(systemStream))
+        .map(entry -> entry.getValue().getWatermarkTime())
+        .min(Long::compare)
+        .orElse(WATERMARK_NOT_EXIST);
+  }
+
+  /* package private for testing */
+  long getWatermarkPerSSP(SystemStreamPartition ssp) {
+    return watermarkStates.get(ssp).getWatermarkTime();
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java
index 6fbc3c1..773f742 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java
@@ -19,6 +19,7 @@
 package org.apache.samza.operators.spec;
 
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.samza.operators.functions.WatermarkFunction;
 import org.apache.samza.system.StreamSpec;
 
 import java.util.function.BiFunction;
@@ -49,4 +50,9 @@ public class InputOperatorSpec<K, V, M> extends OperatorSpec<Pair<K, V>, M> {
   public BiFunction<K, V, M> getMsgBuilder() {
     return this.msgBuilder;
   }
+
+  @Override
+  public WatermarkFunction getWatermarkFn() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/operators/spec/JoinOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/JoinOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/JoinOperatorSpec.java
index 16f59d7..f4fe0fd 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/spec/JoinOperatorSpec.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/JoinOperatorSpec.java
@@ -19,6 +19,7 @@
 package org.apache.samza.operators.spec;
 
 import org.apache.samza.operators.functions.JoinFunction;
+import org.apache.samza.operators.functions.WatermarkFunction;
 
 
 /**
@@ -70,4 +71,9 @@ public class JoinOperatorSpec<K, M, JM, RM> extends OperatorSpec<Object, RM> { /
   public long getTtlMs() {
     return ttlMs;
   }
+
+  @Override
+  public WatermarkFunction getWatermarkFn() {
+    return joinFn instanceof WatermarkFunction ? (WatermarkFunction) joinFn : null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
index f64e123..4047d92 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
@@ -19,6 +19,7 @@
 package org.apache.samza.operators.spec;
 
 import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.operators.functions.WatermarkFunction;
 
 import java.util.Collection;
 import java.util.LinkedHashSet;
@@ -118,4 +119,6 @@ public abstract class OperatorSpec<M, OM> {
   public final String getOpName() {
     return String.format("%s-%s", getOpCode().name().toLowerCase(), getOpId());
   }
+
+  abstract public WatermarkFunction getWatermarkFn();
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/operators/spec/OutputOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/OutputOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/OutputOperatorSpec.java
index e6767ec..9759392 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/spec/OutputOperatorSpec.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/OutputOperatorSpec.java
@@ -19,6 +19,8 @@
 package org.apache.samza.operators.spec;
 
 
+import org.apache.samza.operators.functions.WatermarkFunction;
+
 /**
  * The spec for an operator that outputs a {@link org.apache.samza.operators.MessageStream} to a
  * {@link org.apache.samza.system.SystemStream}.
@@ -52,4 +54,9 @@ public class OutputOperatorSpec<M> extends OperatorSpec<M, Void> {
   public OutputStreamImpl<?, ?, M> getOutputStream() {
     return this.outputStream;
   }
+
+  @Override
+  public WatermarkFunction getWatermarkFn() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java
index 2b55d95..1145be8 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java
@@ -19,6 +19,7 @@
 package org.apache.samza.operators.spec;
 
 import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.operators.functions.WatermarkFunction;
 
 
 /**
@@ -48,4 +49,9 @@ public class SinkOperatorSpec<M> extends OperatorSpec<M, Void> {
   public SinkFunction<M> getSinkFn() {
     return this.sinkFn;
   }
+
+  @Override
+  public WatermarkFunction getWatermarkFn() {
+    return sinkFn instanceof WatermarkFunction ? (WatermarkFunction) sinkFn : null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java
index 1f2f683..aace2e2 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java
@@ -19,6 +19,7 @@
 package org.apache.samza.operators.spec;
 
 import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.functions.WatermarkFunction;
 
 
 /**
@@ -46,4 +47,9 @@ public class StreamOperatorSpec<M, OM> extends OperatorSpec<M, OM> {
   public FlatMapFunction<M, OM> getTransformFn() {
     return this.transformFn;
   }
+
+  @Override
+  public WatermarkFunction getWatermarkFn() {
+    return transformFn instanceof WatermarkFunction ? (WatermarkFunction) transformFn : null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java
index 0937499..75f1427 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java
@@ -19,6 +19,8 @@
 
 package org.apache.samza.operators.spec;
 
+import org.apache.samza.operators.functions.FoldLeftFunction;
+import org.apache.samza.operators.functions.WatermarkFunction;
 import org.apache.samza.operators.triggers.AnyTrigger;
 import org.apache.samza.operators.triggers.RepeatingTrigger;
 import org.apache.samza.operators.triggers.TimeBasedTrigger;
@@ -109,4 +111,10 @@ public class WindowOperatorSpec<M, WK, WV> extends OperatorSpec<M, WindowPane<WK
     }
     return timeBasedTriggers;
   }
+
+  @Override
+  public WatermarkFunction getWatermarkFn() {
+    FoldLeftFunction fn = window.getFoldLeftFunction();
+    return fn instanceof WatermarkFunction ? (WatermarkFunction) fn : null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/serializers/IntermediateMessageSerde.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/serializers/IntermediateMessageSerde.java b/samza-core/src/main/java/org/apache/samza/serializers/IntermediateMessageSerde.java
index 0b98ec6..2ed559f 100644
--- a/samza-core/src/main/java/org/apache/samza/serializers/IntermediateMessageSerde.java
+++ b/samza-core/src/main/java/org/apache/samza/serializers/IntermediateMessageSerde.java
@@ -21,9 +21,9 @@ package org.apache.samza.serializers;
 
 import java.util.Arrays;
 import org.apache.samza.SamzaException;
-import org.apache.samza.message.EndOfStreamMessage;
-import org.apache.samza.message.IntermediateMessageType;
-import org.apache.samza.message.WatermarkMessage;
+import org.apache.samza.system.EndOfStreamMessage;
+import org.apache.samza.system.MessageType;
+import org.apache.samza.system.WatermarkMessage;
 import org.codehaus.jackson.type.TypeReference;
 
 
@@ -86,16 +86,16 @@ public class IntermediateMessageSerde implements Serde<Object> {
   public Object fromBytes(byte[] bytes) {
     try {
       final Object object;
-      final IntermediateMessageType type = IntermediateMessageType.values()[bytes[0]];
+      final MessageType type = MessageType.values()[bytes[0]];
       final byte [] data = Arrays.copyOfRange(bytes, 1, bytes.length);
       switch (type) {
         case USER_MESSAGE:
           object = userMessageSerde.fromBytes(data);
           break;
-        case WATERMARK_MESSAGE:
+        case WATERMARK:
           object = watermarkSerde.fromBytes(data);
           break;
-        case END_OF_STREAM_MESSAGE:
+        case END_OF_STREAM:
           object = eosSerde.fromBytes(data);
           break;
         default:
@@ -117,15 +117,15 @@ public class IntermediateMessageSerde implements Serde<Object> {
   @Override
   public byte[] toBytes(Object object) {
     final byte [] data;
-    final IntermediateMessageType type = IntermediateMessageType.of(object);
+    final MessageType type = MessageType.of(object);
     switch (type) {
       case USER_MESSAGE:
         data = userMessageSerde.toBytes(object);
         break;
-      case WATERMARK_MESSAGE:
+      case WATERMARK:
         data = watermarkSerde.toBytes((WatermarkMessage) object);
         break;
-      case END_OF_STREAM_MESSAGE:
+      case END_OF_STREAM:
         data = eosSerde.toBytes((EndOfStreamMessage) object);
         break;
       default:

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/task/AsyncStreamTaskAdapter.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/task/AsyncStreamTaskAdapter.java b/samza-core/src/main/java/org/apache/samza/task/AsyncStreamTaskAdapter.java
index e57a89f..e2fea95 100644
--- a/samza-core/src/main/java/org/apache/samza/task/AsyncStreamTaskAdapter.java
+++ b/samza-core/src/main/java/org/apache/samza/task/AsyncStreamTaskAdapter.java
@@ -21,11 +21,7 @@ package org.apache.samza.task;
 
 import java.util.concurrent.ExecutorService;
 import org.apache.samza.config.Config;
-import org.apache.samza.control.ControlMessageListenerTask;
-import org.apache.samza.control.Watermark;
-import org.apache.samza.control.IOGraph;
 import org.apache.samza.system.IncomingMessageEnvelope;
-import org.apache.samza.system.SystemStream;
 
 
 /**
@@ -34,7 +30,7 @@ import org.apache.samza.system.SystemStream;
  * the callbacks once it's done. If the thread pool is null, it follows the legacy
  * synchronous model to execute the tasks on the run loop thread.
  */
-public class AsyncStreamTaskAdapter implements AsyncStreamTask, InitableTask, WindowableTask, ClosableTask, EndOfStreamListenerTask, ControlMessageListenerTask {
+public class AsyncStreamTaskAdapter implements AsyncStreamTask, InitableTask, WindowableTask, ClosableTask, EndOfStreamListenerTask {
   private final StreamTask wrappedTask;
   private final ExecutorService executor;
 
@@ -100,20 +96,4 @@ public class AsyncStreamTaskAdapter implements AsyncStreamTask, InitableTask, Wi
       ((EndOfStreamListenerTask) wrappedTask).onEndOfStream(collector, coordinator);
     }
   }
-
-  @Override
-  public IOGraph getIOGraph() {
-    if (wrappedTask instanceof ControlMessageListenerTask) {
-      return ((ControlMessageListenerTask) wrappedTask).getIOGraph();
-    }
-    return null;
-  }
-
-  @Override
-  public void onWatermark(Watermark watermark, SystemStream stream, MessageCollector collector, TaskCoordinator coordinator) {
-    if (wrappedTask instanceof ControlMessageListenerTask) {
-      ((ControlMessageListenerTask) wrappedTask).onWatermark(watermark, stream, collector, coordinator);
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java b/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java
index 16b7e40..0074e24 100644
--- a/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java
+++ b/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java
@@ -21,25 +21,28 @@ package org.apache.samza.task;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.Config;
-import org.apache.samza.control.ControlMessageListenerTask;
-import org.apache.samza.control.Watermark;
+import org.apache.samza.system.EndOfStreamMessage;
+import org.apache.samza.system.MessageType;
 import org.apache.samza.operators.ContextManager;
 import org.apache.samza.operators.StreamGraphImpl;
 import org.apache.samza.operators.impl.InputOperatorImpl;
 import org.apache.samza.operators.impl.OperatorImplGraph;
-import org.apache.samza.control.IOGraph;
 import org.apache.samza.runtime.ApplicationRunner;
 import org.apache.samza.system.IncomingMessageEnvelope;
 import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.WatermarkMessage;
 import org.apache.samza.util.Clock;
 import org.apache.samza.util.SystemClock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
  * A {@link StreamTask} implementation that brings all the operator API implementation components together and
  * feeds the input messages into the user-defined transformation chains in {@link StreamApplication}.
  */
-public final class StreamOperatorTask implements StreamTask, InitableTask, WindowableTask, ClosableTask, ControlMessageListenerTask {
+public final class StreamOperatorTask implements StreamTask, InitableTask, WindowableTask, ClosableTask {
+  private static final Logger LOG = LoggerFactory.getLogger(StreamOperatorTask.class);
 
   private final StreamApplication streamApplication;
   private final ApplicationRunner runner;
@@ -47,7 +50,6 @@ public final class StreamOperatorTask implements StreamTask, InitableTask, Windo
 
   private OperatorImplGraph operatorImplGraph;
   private ContextManager contextManager;
-  private IOGraph ioGraph;
 
   /**
    * Constructs an adaptor task to run the user-implemented {@link StreamApplication}.
@@ -91,7 +93,6 @@ public final class StreamOperatorTask implements StreamTask, InitableTask, Windo
 
     // create the operator impl DAG corresponding to the logical operator spec DAG
     this.operatorImplGraph = new OperatorImplGraph(streamGraph, config, context, clock);
-    this.ioGraph = streamGraph.toIOGraph();
   }
 
   /**
@@ -110,7 +111,21 @@ public final class StreamOperatorTask implements StreamTask, InitableTask, Windo
     SystemStream systemStream = ime.getSystemStreamPartition().getSystemStream();
     InputOperatorImpl inputOpImpl = operatorImplGraph.getInputOperator(systemStream);
     if (inputOpImpl != null) {
-      inputOpImpl.onMessage(Pair.of(ime.getKey(), ime.getMessage()), collector, coordinator);
+      switch (MessageType.of(ime.getMessage())) {
+        case USER_MESSAGE:
+          inputOpImpl.onMessage(Pair.of(ime.getKey(), ime.getMessage()), collector, coordinator);
+          break;
+
+        case END_OF_STREAM:
+          EndOfStreamMessage eosMessage = (EndOfStreamMessage) ime.getMessage();
+          inputOpImpl.aggregateEndOfStream(eosMessage, ime.getSystemStreamPartition(), collector, coordinator);
+          break;
+
+        case WATERMARK:
+          WatermarkMessage watermarkMessage = (WatermarkMessage) ime.getMessage();
+          inputOpImpl.aggregateWatermark(watermarkMessage, ime.getSystemStreamPartition(), collector, coordinator);
+          break;
+      }
     }
   }
 
@@ -121,22 +136,6 @@ public final class StreamOperatorTask implements StreamTask, InitableTask, Windo
   }
 
   @Override
-  public IOGraph getIOGraph() {
-    return ioGraph;
-  }
-
-  @Override
-  public final void onWatermark(Watermark watermark,
-      SystemStream systemStream,
-      MessageCollector collector,
-      TaskCoordinator coordinator) {
-    InputOperatorImpl inputOpImpl = operatorImplGraph.getInputOperator(systemStream);
-    if (inputOpImpl != null) {
-      inputOpImpl.onWatermark(watermark, collector, coordinator);
-    }
-  }
-
-  @Override
   public void close() throws Exception {
     if (this.contextManager != null) {
       this.contextManager.close();

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/scala/org/apache/samza/checkpoint/OffsetManager.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/checkpoint/OffsetManager.scala b/samza-core/src/main/scala/org/apache/samza/checkpoint/OffsetManager.scala
index 8c739d4..1b2ce80 100644
--- a/samza-core/src/main/scala/org/apache/samza/checkpoint/OffsetManager.scala
+++ b/samza-core/src/main/scala/org/apache/samza/checkpoint/OffsetManager.scala
@@ -24,6 +24,7 @@ package org.apache.samza.checkpoint
 import java.util.HashMap
 import java.util.concurrent.ConcurrentHashMap
 
+import org.apache.samza.system.IncomingMessageEnvelope
 import org.apache.samza.SamzaException
 import org.apache.samza.config.Config
 import org.apache.samza.config.StreamConfig.Config2Stream
@@ -193,7 +194,7 @@ class OffsetManager(
    */
   def update(taskName: TaskName, systemStreamPartition: SystemStreamPartition, offset: String) {
     lastProcessedOffsets.putIfAbsent(taskName, new ConcurrentHashMap[SystemStreamPartition, String]())
-    if (offset != null) {
+    if (offset != null && !offset.equals(IncomingMessageEnvelope.END_OF_STREAM_OFFSET)) {
       lastProcessedOffsets.get(taskName).put(systemStreamPartition, offset)
     }
   }
@@ -216,6 +217,10 @@ class OffsetManager(
     }
   }
 
+  def setStartingOffset(taskName: TaskName, ssp: SystemStreamPartition, offset: String): Unit = {
+    startingOffsets += taskName -> (startingOffsets(taskName) + (ssp -> offset))
+  }
+
   /**
     * Gets a snapshot of all the current offsets for the specified task. This is useful to
     * ensure there are no concurrent updates to the offsets between when this method is

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala b/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala
index 5a19d90..acec365 100644
--- a/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala
+++ b/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala
@@ -20,25 +20,17 @@
 package org.apache.samza.container
 
 
-import com.google.common.collect.HashMultimap
-import com.google.common.collect.Multimap
 import org.apache.samza.SamzaException
 import org.apache.samza.checkpoint.OffsetManager
 import org.apache.samza.config.Config
 import org.apache.samza.config.StreamConfig.Config2Stream
-import org.apache.samza.control.ControlMessageListenerTask
-import org.apache.samza.control.ControlMessageUtils
-import org.apache.samza.control.EndOfStreamManager
-import org.apache.samza.control.WatermarkManager
 import org.apache.samza.job.model.JobModel
-import org.apache.samza.message.MessageType
 import org.apache.samza.metrics.MetricsReporter
 import org.apache.samza.storage.TaskStorageManager
 import org.apache.samza.system.IncomingMessageEnvelope
 import org.apache.samza.system.StreamMetadataCache
 import org.apache.samza.system.SystemAdmin
 import org.apache.samza.system.SystemConsumers
-import org.apache.samza.system.SystemStream
 import org.apache.samza.system.SystemStreamPartition
 import org.apache.samza.task.AsyncStreamTask
 import org.apache.samza.task.ClosableTask
@@ -47,34 +39,11 @@ import org.apache.samza.task.InitableTask
 import org.apache.samza.task.ReadableCoordinator
 import org.apache.samza.task.StreamTask
 import org.apache.samza.task.TaskCallbackFactory
-import org.apache.samza.task.TaskContext
 import org.apache.samza.task.TaskInstanceCollector
 import org.apache.samza.task.WindowableTask
 import org.apache.samza.util.Logging
 
 import scala.collection.JavaConverters._
-import scala.collection.JavaConversions._
-
-object TaskInstance {
-  /**
-   * Build a map from a stream to its consumer tasks
-   * @param jobModel job model which contains ssp-to-task assignment
-   * @return the map of input stream to tasks
-   */
-  def buildInputToTasks(jobModel: JobModel): Multimap[SystemStream, String] = {
-    val streamToTasks: Multimap[SystemStream, String] = HashMultimap.create[SystemStream, String]
-    if (jobModel != null) {
-      for (containerModel <- jobModel.getContainers.values) {
-        for (taskModel <- containerModel.getTasks.values) {
-          for (ssp <- taskModel.getSystemStreamPartitions) {
-            streamToTasks.put(ssp.getSystemStream, taskModel.getTaskName.toString)
-          }
-        }
-      }
-    }
-    return streamToTasks
-  }
-}
 
 class TaskInstance(
   val task: Any,
@@ -97,35 +66,10 @@ class TaskInstance(
   val isEndOfStreamListenerTask = task.isInstanceOf[EndOfStreamListenerTask]
   val isClosableTask = task.isInstanceOf[ClosableTask]
   val isAsyncTask = task.isInstanceOf[AsyncStreamTask]
-  val isControlMessageListener = task.isInstanceOf[ControlMessageListenerTask]
-
-  val context = new TaskContext {
-    var userContext: Object = null;
-    def getMetricsRegistry = metrics.registry
-    def getSystemStreamPartitions = systemStreamPartitions.asJava
-    def getStore(storeName: String) = if (storageManager != null) {
-      storageManager(storeName)
-    } else {
-      warn("No store found for name: %s" format storeName)
 
-      null
-    }
-    def getTaskName = taskName
-    def getSamzaContainerContext = containerContext
+  val context = new TaskContextImpl(taskName,metrics, containerContext, systemStreamPartitions.asJava, offsetManager,
+                                    storageManager, jobModel, streamMetadataCache)
 
-    override def setStartingOffset(ssp: SystemStreamPartition, offset: String): Unit = {
-      val startingOffsets = offsetManager.startingOffsets
-      offsetManager.startingOffsets += taskName -> (startingOffsets(taskName) + (ssp -> offset))
-    }
-
-    override def setUserContext(context: Object): Unit = {
-      userContext = context
-    }
-
-    override def getUserContext: Object = {
-      userContext
-    }
-  }
   // store the (ssp -> if this ssp is catched up) mapping. "catched up"
   // means the same ssp in other taskInstances have the same offset as
   // the one here.
@@ -133,10 +77,6 @@ class TaskInstance(
     scala.collection.mutable.Map[SystemStreamPartition, Boolean]()
   systemStreamPartitions.foreach(ssp2CaughtupMapping += _ -> false)
 
-  val inputToTasksMapping = TaskInstance.buildInputToTasks(jobModel)
-  var endOfStreamManager: EndOfStreamManager = null
-  var watermarkManager: WatermarkManager = null
-
   val hasIntermediateStreams = config.getStreamIds.exists(config.getIsIntermediate(_))
 
   def registerMetrics {
@@ -169,22 +109,6 @@ class TaskInstance(
     } else {
       debug("Skipping task initialization for taskName: %s" format taskName)
     }
-
-    if (isControlMessageListener) {
-      endOfStreamManager = new EndOfStreamManager(taskName.getTaskName,
-                                                  task.asInstanceOf[ControlMessageListenerTask],
-                                                  inputToTasksMapping,
-                                                  systemStreamPartitions.asJava,
-                                                  streamMetadataCache,
-                                                  collector)
-
-      watermarkManager = new WatermarkManager(taskName.getTaskName,
-                                                  task.asInstanceOf[ControlMessageListenerTask],
-                                                  inputToTasksMapping,
-                                                  systemStreamPartitions.asJava,
-                                                  streamMetadataCache,
-                                                  collector)
-    }
   }
 
   def registerProducers {
@@ -223,51 +147,20 @@ class TaskInstance(
       trace("Processing incoming message envelope for taskName and SSP: %s, %s"
         format (taskName, envelope.getSystemStreamPartition))
 
-      MessageType.of(envelope.getMessage) match {
-        case MessageType.USER_MESSAGE =>
-          if (isAsyncTask) {
-            exceptionHandler.maybeHandle {
-             val callback = callbackFactory.createCallback()
-             task.asInstanceOf[AsyncStreamTask].processAsync(envelope, collector, coordinator, callback)
-            }
-          }
-          else {
-            exceptionHandler.maybeHandle {
-             task.asInstanceOf[StreamTask].process(envelope, collector, coordinator)
-            }
-
-            trace("Updating offset map for taskName, SSP and offset: %s, %s, %s"
-              format(taskName, envelope.getSystemStreamPartition, envelope.getOffset))
-
-            offsetManager.update(taskName, envelope.getSystemStreamPartition, envelope.getOffset)
-          }
+      if (isAsyncTask) {
+        exceptionHandler.maybeHandle {
+          val callback = callbackFactory.createCallback()
+          task.asInstanceOf[AsyncStreamTask].processAsync(envelope, collector, coordinator, callback)
+        }
+      } else {
+        exceptionHandler.maybeHandle {
+         task.asInstanceOf[StreamTask].process(envelope, collector, coordinator)
+        }
 
-        case MessageType.END_OF_STREAM =>
-          if (isControlMessageListener) {
-            // handle eos synchronously.
-            runSync(callbackFactory) {
-              endOfStreamManager.update(envelope, coordinator)
-            }
-          } else {
-            warn("Ignore end-of-stream message due to %s not implementing ControlMessageListener."
-              format(task.getClass.toString))
-          }
+        trace("Updating offset map for taskName, SSP and offset: %s, %s, %s"
+          format (taskName, envelope.getSystemStreamPartition, envelope.getOffset))
 
-        case MessageType.WATERMARK =>
-          if (isControlMessageListener) {
-            // handle watermark synchronously in the run loop thread.
-            // we might consider running it asynchronously later
-            runSync(callbackFactory) {
-              val watermark = watermarkManager.update(envelope)
-              if (watermark != null) {
-                val stream = envelope.getSystemStreamPartition.getSystemStream
-                task.asInstanceOf[ControlMessageListenerTask].onWatermark(watermark, stream, collector, coordinator)
-              }
-            }
-          } else {
-            warn("Ignore watermark message due to %s not implementing ControlMessageListener."
-              format(task.getClass.toString))
-          }
+        offsetManager.update(taskName, envelope.getSystemStreamPartition, envelope.getOffset)
       }
     }
   }
@@ -343,38 +236,32 @@ class TaskInstance(
    * it's already catched-up.
    */
   private def checkCaughtUp(envelope: IncomingMessageEnvelope) = {
-    systemAdmins match {
-      case null => {
-        warn("systemAdmin is null. Set all SystemStreamPartitions to catched-up")
-        ssp2CaughtupMapping(envelope.getSystemStreamPartition) = true
-      }
-      case others => {
-        val startingOffset = offsetManager.getStartingOffset(taskName, envelope.getSystemStreamPartition)
-            .getOrElse(throw new SamzaException("No offset defined for SystemStreamPartition: %s" format envelope.getSystemStreamPartition))
-        val system = envelope.getSystemStreamPartition.getSystem
-        others(system).offsetComparator(envelope.getOffset, startingOffset) match {
-          case null => {
-            info("offsets in " + system + " is not comparable. Set all SystemStreamPartitions to catched-up")
-            ssp2CaughtupMapping(envelope.getSystemStreamPartition) = true // not comparable
-          }
-          case result => {
-            if (result >= 0) {
-              info(envelope.getSystemStreamPartition.toString + " is catched up.")
-              ssp2CaughtupMapping(envelope.getSystemStreamPartition) = true
+    if (IncomingMessageEnvelope.END_OF_STREAM_OFFSET.equals(envelope.getOffset)) {
+      ssp2CaughtupMapping(envelope.getSystemStreamPartition) = true
+    } else {
+      systemAdmins match {
+        case null => {
+          warn("systemAdmin is null. Set all SystemStreamPartitions to catched-up")
+          ssp2CaughtupMapping(envelope.getSystemStreamPartition) = true
+        }
+        case others => {
+          val startingOffset = offsetManager.getStartingOffset(taskName, envelope.getSystemStreamPartition)
+              .getOrElse(throw new SamzaException("No offset defined for SystemStreamPartition: %s" format envelope.getSystemStreamPartition))
+          val system = envelope.getSystemStreamPartition.getSystem
+          others(system).offsetComparator(envelope.getOffset, startingOffset) match {
+            case null => {
+              info("offsets in " + system + " is not comparable. Set all SystemStreamPartitions to catched-up")
+              ssp2CaughtupMapping(envelope.getSystemStreamPartition) = true // not comparable
+            }
+            case result => {
+              if (result >= 0) {
+                info(envelope.getSystemStreamPartition.toString + " is catched up.")
+                ssp2CaughtupMapping(envelope.getSystemStreamPartition) = true
+              }
             }
           }
         }
       }
     }
   }
-
-  private def runSync(callbackFactory: TaskCallbackFactory)(runCodeBlock: => Unit) = {
-    val callback = callbackFactory.createCallback()
-    try {
-      runCodeBlock
-      callback.complete()
-    } catch {
-      case t: Throwable => callback.failure(t)
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/scala/org/apache/samza/serializers/SerdeManager.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/serializers/SerdeManager.scala b/samza-core/src/main/scala/org/apache/samza/serializers/SerdeManager.scala
index 0100c78..76594ae 100644
--- a/samza-core/src/main/scala/org/apache/samza/serializers/SerdeManager.scala
+++ b/samza-core/src/main/scala/org/apache/samza/serializers/SerdeManager.scala
@@ -20,12 +20,13 @@
 package org.apache.samza.serializers
 
 import org.apache.samza.SamzaException
-import org.apache.samza.message.ControlMessage
-import org.apache.samza.message.WatermarkMessage
+import org.apache.samza.system.ControlMessage
 import org.apache.samza.system.SystemStream
 import org.apache.samza.system.OutgoingMessageEnvelope
 import org.apache.samza.system.IncomingMessageEnvelope
 import org.apache.samza.config.StorageConfig
+import org.apache.samza.system.WatermarkMessage
+
 
 class SerdeManager(
   serdes: Map[String, Serde[Object]] = Map(),

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/test/java/org/apache/samza/control/TestControlMessageUtils.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/control/TestControlMessageUtils.java b/samza-core/src/test/java/org/apache/samza/control/TestControlMessageUtils.java
deleted file mode 100644
index 8351802..0000000
--- a/samza-core/src/test/java/org/apache/samza/control/TestControlMessageUtils.java
+++ /dev/null
@@ -1,115 +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.control;
-
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Multimap;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.apache.samza.Partition;
-import org.apache.samza.container.TaskName;
-import org.apache.samza.message.ControlMessage;
-import org.apache.samza.system.OutgoingMessageEnvelope;
-import org.apache.samza.system.StreamMetadataCache;
-import org.apache.samza.system.StreamSpec;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.system.SystemStreamMetadata;
-import org.apache.samza.task.MessageCollector;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyBoolean;
-import static org.mockito.Matchers.anyObject;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-
-public class TestControlMessageUtils {
-
-  @Test
-  public void testSendControlMessage() {
-    SystemStreamMetadata metadata = mock(SystemStreamMetadata.class);
-    Map<Partition, SystemStreamMetadata.SystemStreamPartitionMetadata> partitionMetadata = new HashMap<>();
-    partitionMetadata.put(new Partition(0), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
-    partitionMetadata.put(new Partition(1), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
-    partitionMetadata.put(new Partition(2), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
-    partitionMetadata.put(new Partition(3), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
-    when(metadata.getSystemStreamPartitionMetadata()).thenReturn(partitionMetadata);
-    StreamMetadataCache metadataCache = mock(StreamMetadataCache.class);
-    when(metadataCache.getSystemStreamMetadata(anyObject(), anyBoolean())).thenReturn(metadata);
-
-    SystemStream systemStream = new SystemStream("test-system", "test-stream");
-    Set<Integer> partitions = new HashSet<>();
-    MessageCollector collector = mock(MessageCollector.class);
-    doAnswer(invocation -> {
-        OutgoingMessageEnvelope envelope = (OutgoingMessageEnvelope) invocation.getArguments()[0];
-        partitions.add((Integer) envelope.getPartitionKey());
-        assertEquals(envelope.getSystemStream(), systemStream);
-        return null;
-      }).when(collector).send(any());
-
-    ControlMessageUtils.sendControlMessage(mock(ControlMessage.class), systemStream, metadataCache, collector);
-    assertEquals(partitions.size(), 4);
-  }
-
-  @Test
-  public void testCalculateUpstreamTaskCounts() {
-    SystemStream input1 = new SystemStream("test-system", "input-stream-1");
-    SystemStream input2 = new SystemStream("test-system", "input-stream-2");
-    SystemStream input3 = new SystemStream("test-system", "input-stream-3");
-
-    Multimap<SystemStream, String> inputToTasks = HashMultimap.create();
-    TaskName t0 = new TaskName("task 0"); //consume input1 and input2
-    TaskName t1 = new TaskName("task 1"); //consume input1 and input2 and input 3
-    TaskName t2 = new TaskName("task 2"); //consume input2 and input 3
-    inputToTasks.put(input1, t0.getTaskName());
-    inputToTasks.put(input1, t1.getTaskName());
-    inputToTasks.put(input2, t0.getTaskName());
-    inputToTasks.put(input2, t1.getTaskName());
-    inputToTasks.put(input2, t2.getTaskName());
-    inputToTasks.put(input3, t1.getTaskName());
-    inputToTasks.put(input3, t2.getTaskName());
-
-    StreamSpec inputSpec2 = new StreamSpec("input-stream-2", "input-stream-2", "test-system");
-    StreamSpec inputSpec3 = new StreamSpec("input-stream-3", "input-stream-3", "test-system");
-    StreamSpec intSpec1 = new StreamSpec("int-stream-1", "int-stream-1", "test-system");
-    StreamSpec intSpec2 = new StreamSpec("int-stream-2", "int-stream-2", "test-system");
-
-    List<IOGraph.IONode> nodes = new ArrayList<>();
-    IOGraph.IONode node = new IOGraph.IONode(intSpec1, true);
-    node.addInput(inputSpec2);
-    nodes.add(node);
-    node = new IOGraph.IONode(intSpec2, true);
-    node.addInput(inputSpec3);
-    nodes.add(node);
-    IOGraph ioGraph = new IOGraph(nodes);
-
-    Map<SystemStream, Integer> counts = ControlMessageUtils.calculateUpstreamTaskCounts(inputToTasks, ioGraph);
-    assertEquals(counts.get(intSpec1.toSystemStream()).intValue(), 3);
-    assertEquals(counts.get(intSpec2.toSystemStream()).intValue(), 2);
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/test/java/org/apache/samza/control/TestEndOfStreamManager.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/control/TestEndOfStreamManager.java b/samza-core/src/test/java/org/apache/samza/control/TestEndOfStreamManager.java
deleted file mode 100644
index cc70b6b..0000000
--- a/samza-core/src/test/java/org/apache/samza/control/TestEndOfStreamManager.java
+++ /dev/null
@@ -1,333 +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.control;
-
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Multimap;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.apache.samza.Partition;
-import org.apache.samza.container.TaskName;
-import org.apache.samza.message.EndOfStreamMessage;
-import org.apache.samza.operators.spec.OperatorSpecs;
-import org.apache.samza.operators.spec.OutputOperatorSpec;
-import org.apache.samza.operators.spec.OutputStreamImpl;
-import org.apache.samza.system.IncomingMessageEnvelope;
-import org.apache.samza.system.OutgoingMessageEnvelope;
-import org.apache.samza.system.StreamMetadataCache;
-import org.apache.samza.system.StreamSpec;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.system.SystemStreamMetadata;
-import org.apache.samza.system.SystemStreamPartition;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.ArgumentCaptor;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyBoolean;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Matchers.anyObject;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.doNothing;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-
-public class TestEndOfStreamManager {
-  StreamMetadataCache metadataCache;
-
-  @Before
-  public void setup() {
-    SystemStreamMetadata metadata = mock(SystemStreamMetadata.class);
-    Map<Partition, SystemStreamMetadata.SystemStreamPartitionMetadata> partitionMetadata = new HashMap<>();
-    partitionMetadata.put(new Partition(0), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
-    partitionMetadata.put(new Partition(1), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
-    partitionMetadata.put(new Partition(2), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
-    partitionMetadata.put(new Partition(3), mock(SystemStreamMetadata.SystemStreamPartitionMetadata.class));
-    when(metadata.getSystemStreamPartitionMetadata()).thenReturn(partitionMetadata);
-    metadataCache = mock(StreamMetadataCache.class);
-    when(metadataCache.getSystemStreamMetadata(anyObject(), anyBoolean())).thenReturn(metadata);
-  }
-
-  @Test
-  public void testUpdateFromInputSource() {
-    SystemStreamPartition ssp = new SystemStreamPartition("test-system", "test-stream", new Partition(0));
-    TaskName taskName = new TaskName("Task 0");
-    Multimap<SystemStream, String> streamToTasks = HashMultimap.create();
-    streamToTasks.put(ssp.getSystemStream(), taskName.getTaskName());
-    ControlMessageListenerTask listener = mock(ControlMessageListenerTask.class);
-    when(listener.getIOGraph()).thenReturn(new IOGraph(Collections.emptyList()));
-    EndOfStreamManager manager = new EndOfStreamManager("Task 0", listener, streamToTasks, Collections.singleton(ssp), null, null);
-    manager.update(EndOfStreamManager.buildEndOfStreamEnvelope(ssp), mock(TaskCoordinator.class));
-    assertTrue(manager.isEndOfStream(ssp.getSystemStream()));
-  }
-
-  @Test
-  public void testUpdateFromIntermediateStream() {
-    SystemStreamPartition[] ssps = new SystemStreamPartition[3];
-    ssps[0] = new SystemStreamPartition("test-system", "test-stream-1", new Partition(0));
-    ssps[1] = new SystemStreamPartition("test-system", "test-stream-2", new Partition(0));
-    ssps[2] = new SystemStreamPartition("test-system", "test-stream-2", new Partition(1));
-
-    TaskName taskName = new TaskName("Task 0");
-    Multimap<SystemStream, String> streamToTasks = HashMultimap.create();
-    for (SystemStreamPartition ssp : ssps) {
-      streamToTasks.put(ssp.getSystemStream(), taskName.getTaskName());
-    }
-    ControlMessageListenerTask listener = mock(ControlMessageListenerTask.class);
-    when(listener.getIOGraph()).thenReturn(new IOGraph(Collections.emptyList()));
-    EndOfStreamManager manager = new EndOfStreamManager("Task 0", listener, streamToTasks, new HashSet<>(Arrays.asList(ssps)), null, null);
-
-    int envelopeCount = 4;
-    IncomingMessageEnvelope[] envelopes = new IncomingMessageEnvelope[envelopeCount];
-    for (int i = 0; i < envelopeCount; i++) {
-      envelopes[i] = new IncomingMessageEnvelope(ssps[0], "dummy-offset", "", new EndOfStreamMessage("task " + i, envelopeCount));
-    }
-    TaskCoordinator coordinator = mock(TaskCoordinator.class);
-
-    // verify the first three messages won't result in end-of-stream
-    for (int i = 0; i < 3; i++) {
-      manager.update(envelopes[i], coordinator);
-      assertFalse(manager.isEndOfStream(ssps[0].getSystemStream()));
-    }
-    // the fourth message will end the stream
-    manager.update(envelopes[3], coordinator);
-    assertTrue(manager.isEndOfStream(ssps[0].getSystemStream()));
-    assertFalse(manager.isEndOfStream(ssps[1].getSystemStream()));
-
-    // stream2 has two partitions assigned to this task, so it requires a message from each partition to end it
-    envelopes = new IncomingMessageEnvelope[envelopeCount];
-    for (int i = 0; i < envelopeCount; i++) {
-      envelopes[i] = new IncomingMessageEnvelope(ssps[1], "dummy-offset", "dummy-key", new EndOfStreamMessage("task " + i, envelopeCount));
-    }
-    // verify the messages for the partition 0 won't result in end-of-stream
-    for (int i = 0; i < 4; i++) {
-      manager.update(envelopes[i], coordinator);
-      assertFalse(manager.isEndOfStream(ssps[1].getSystemStream()));
-    }
-    for (int i = 0; i < envelopeCount; i++) {
-      envelopes[i] = new IncomingMessageEnvelope(ssps[2], "dummy-offset", "dummy-key", new EndOfStreamMessage("task " + i, envelopeCount));
-    }
-    for (int i = 0; i < 3; i++) {
-      manager.update(envelopes[i], coordinator);
-      assertFalse(manager.isEndOfStream(ssps[1].getSystemStream()));
-    }
-    // the fourth message will end the stream
-    manager.update(envelopes[3], coordinator);
-    assertTrue(manager.isEndOfStream(ssps[1].getSystemStream()));
-  }
-
-  @Test
-  public void testUpdateFromIntermediateStreamWith2Tasks() {
-    SystemStreamPartition[] ssps0 = new SystemStreamPartition[2];
-    ssps0[0] = new SystemStreamPartition("test-system", "test-stream-1", new Partition(0));
-    ssps0[1] = new SystemStreamPartition("test-system", "test-stream-2", new Partition(0));
-
-    SystemStreamPartition ssp1 = new SystemStreamPartition("test-system", "test-stream-2", new Partition(1));
-
-    TaskName t0 = new TaskName("Task 0");
-    Multimap<SystemStream, String> streamToTasks = HashMultimap.create();
-    for (SystemStreamPartition ssp : ssps0) {
-      streamToTasks.put(ssp.getSystemStream(), t0.getTaskName());
-    }
-
-    TaskName t1 = new TaskName("Task 1");
-    streamToTasks.put(ssp1, t1.getTaskName());
-
-    List<StreamSpec> inputs = new ArrayList<>();
-    inputs.add(new StreamSpec("test-stream-1", "test-stream-1", "test-system"));
-    inputs.add(new StreamSpec("test-stream-2", "test-stream-2", "test-system"));
-    StreamSpec outputSpec = new StreamSpec("int-stream", "int-stream", "test-system");
-    IOGraph ioGraph = TestIOGraph.buildSimpleIOGraph(inputs, outputSpec, true);
-
-    ControlMessageListenerTask listener = mock(ControlMessageListenerTask.class);
-    when(listener.getIOGraph()).thenReturn(ioGraph);
-
-    EndOfStreamManager manager0 = spy(new EndOfStreamManager("Task 0", listener, streamToTasks, new HashSet<>(Arrays.asList(ssps0)), null, null));
-    manager0.update(EndOfStreamManager.buildEndOfStreamEnvelope(ssps0[0]), mock(TaskCoordinator.class));
-    assertTrue(manager0.isEndOfStream(ssps0[0].getSystemStream()));
-    doNothing().when(manager0).sendEndOfStream(any(), anyInt());
-    manager0.update(EndOfStreamManager.buildEndOfStreamEnvelope(ssps0[1]), mock(TaskCoordinator.class));
-    assertTrue(manager0.isEndOfStream(ssps0[1].getSystemStream()));
-    verify(manager0).sendEndOfStream(any(), anyInt());
-
-    EndOfStreamManager manager1 = spy(new EndOfStreamManager("Task 1", listener, streamToTasks, Collections.singleton(
-        ssp1), null, null));
-    doNothing().when(manager1).sendEndOfStream(any(), anyInt());
-    manager1.update(EndOfStreamManager.buildEndOfStreamEnvelope(ssp1), mock(TaskCoordinator.class));
-    assertTrue(manager1.isEndOfStream(ssp1.getSystemStream()));
-    verify(manager1).sendEndOfStream(any(), anyInt());
-  }
-
-  @Test
-  public void testSendEndOfStream() {
-    StreamSpec ints = new StreamSpec("int-stream", "int-stream", "test-system");
-    StreamSpec input = new StreamSpec("input-stream", "input-stream", "test-system");
-    IOGraph ioGraph = TestIOGraph.buildSimpleIOGraph(Collections.singletonList(input), ints, true);
-
-    Multimap<SystemStream, String> inputToTasks = HashMultimap.create();
-    for (int i = 0; i < 8; i++) {
-      inputToTasks.put(input.toSystemStream(), "Task " + i);
-    }
-
-    MessageCollector collector = mock(MessageCollector.class);
-    TaskName taskName = new TaskName("Task 0");
-    ControlMessageListenerTask listener = mock(ControlMessageListenerTask.class);
-    when(listener.getIOGraph()).thenReturn(ioGraph);
-    EndOfStreamManager manager = new EndOfStreamManager(taskName.getTaskName(),
-        listener,
-        inputToTasks,
-        Collections.EMPTY_SET,
-        metadataCache,
-        collector);
-
-    Set<Integer> partitions = new HashSet<>();
-    doAnswer(invocation -> {
-        OutgoingMessageEnvelope envelope = (OutgoingMessageEnvelope) invocation.getArguments()[0];
-        partitions.add((Integer) envelope.getPartitionKey());
-        EndOfStreamMessage eosMessage = (EndOfStreamMessage) envelope.getMessage();
-        assertEquals(eosMessage.getTaskName(), taskName.getTaskName());
-        assertEquals(eosMessage.getTaskCount(), 8);
-        return null;
-      }).when(collector).send(any());
-
-    manager.sendEndOfStream(input.toSystemStream(), 8);
-    assertEquals(partitions.size(), 4);
-  }
-
-  @Test
-  public void testPropagate() {
-    List<StreamSpec> inputs = new ArrayList<>();
-    inputs.add(new StreamSpec("input-stream-1", "input-stream-1", "test-system"));
-    inputs.add(new StreamSpec("input-stream-2", "input-stream-2", "test-system"));
-    StreamSpec outputSpec = new StreamSpec("int-stream", "int-stream", "test-system");
-
-    SystemStream input1 = new SystemStream("test-system", "input-stream-1");
-    SystemStream input2 = new SystemStream("test-system", "input-stream-2");
-    SystemStream ints = new SystemStream("test-system", "int-stream");
-    SystemStreamPartition[] ssps = new SystemStreamPartition[3];
-    ssps[0] = new SystemStreamPartition(input1, new Partition(0));
-    ssps[1] = new SystemStreamPartition(input2, new Partition(0));
-    ssps[2] = new SystemStreamPartition(ints, new Partition(0));
-
-    Set<SystemStreamPartition> sspSet = new HashSet<>(Arrays.asList(ssps));
-    TaskName taskName = new TaskName("task 0");
-    Multimap<SystemStream, String> streamToTasks = HashMultimap.create();
-    for (SystemStreamPartition ssp : ssps) {
-      streamToTasks.put(ssp.getSystemStream(), taskName.getTaskName());
-    }
-
-    IOGraph ioGraph = TestIOGraph.buildSimpleIOGraph(inputs, outputSpec, true);
-    MessageCollector collector = mock(MessageCollector.class);
-    ControlMessageListenerTask listener = mock(ControlMessageListenerTask.class);
-    when(listener.getIOGraph()).thenReturn(ioGraph);
-    EndOfStreamManager manager = spy(
-        new EndOfStreamManager("task 0", listener, streamToTasks, sspSet, metadataCache, collector));
-    TaskCoordinator coordinator = mock(TaskCoordinator.class);
-
-    // ssp1 end-of-stream, wait for ssp2
-    manager.update(EndOfStreamManager.buildEndOfStreamEnvelope(ssps[0]), coordinator);
-    verify(manager, never()).sendEndOfStream(any(), anyInt());
-
-    // ssp2 end-of-stream, propagate to intermediate
-    manager.update(EndOfStreamManager.buildEndOfStreamEnvelope(ssps[1]), coordinator);
-    doNothing().when(manager).sendEndOfStream(any(), anyInt());
-    ArgumentCaptor<SystemStream> argument = ArgumentCaptor.forClass(SystemStream.class);
-    verify(manager).sendEndOfStream(argument.capture(), anyInt());
-    assertEquals(ints, argument.getValue());
-
-    // intermediate end-of-stream, shutdown the task
-    manager.update(EndOfStreamManager.buildEndOfStreamEnvelope(ssps[2]), coordinator);
-    doNothing().when(coordinator).shutdown(any());
-    ArgumentCaptor<TaskCoordinator.RequestScope> arg = ArgumentCaptor.forClass(TaskCoordinator.RequestScope.class);
-    verify(coordinator).shutdown(arg.capture());
-    assertEquals(TaskCoordinator.RequestScope.CURRENT_TASK, arg.getValue());
-  }
-
-  //  Test the case when the publishing tasks to intermediate stream is a subset of total tasks
-  @Test
-  public void testPropogateWith2Tasks() {
-    StreamSpec outputSpec = new StreamSpec("int-stream", "int-stream", "test-system");
-    OutputStreamImpl outputStream = new OutputStreamImpl(outputSpec, null, null);
-    OutputOperatorSpec partitionByOp = OperatorSpecs.createPartitionByOperatorSpec(outputStream, 0);
-
-    List<StreamSpec> inputs = new ArrayList<>();
-    inputs.add(new StreamSpec("input-stream-1", "input-stream-1", "test-system"));
-
-    IOGraph ioGraph = TestIOGraph.buildSimpleIOGraph(inputs, outputSpec, true);
-
-    SystemStream input1 = new SystemStream("test-system", "input-stream-1");
-    SystemStream ints = new SystemStream("test-system", "int-stream");
-    SystemStreamPartition ssp1 = new SystemStreamPartition(input1, new Partition(0));
-    SystemStreamPartition ssp2 = new SystemStreamPartition(ints, new Partition(0));
-
-    TaskName t0 = new TaskName("task 0");
-    TaskName t1 = new TaskName("task 1");
-    Multimap<SystemStream, String> streamToTasks = HashMultimap.create();
-    streamToTasks.put(ssp1.getSystemStream(), t0.getTaskName());
-    streamToTasks.put(ssp2.getSystemStream(), t1.getTaskName());
-
-    ControlMessageListenerTask listener = mock(ControlMessageListenerTask.class);
-    when(listener.getIOGraph()).thenReturn(ioGraph);
-
-    EndOfStreamManager manager0 = spy(
-        new EndOfStreamManager(t0.getTaskName(), listener, streamToTasks, Collections.singleton(ssp1), metadataCache, null));
-    EndOfStreamManager manager1 = spy(
-        new EndOfStreamManager(t1.getTaskName(), listener, streamToTasks, Collections.singleton(ssp2), metadataCache, null));
-
-    TaskCoordinator coordinator0 = mock(TaskCoordinator.class);
-    TaskCoordinator coordinator1 = mock(TaskCoordinator.class);
-
-    // ssp1 end-of-stream
-    doNothing().when(manager0).sendEndOfStream(any(), anyInt());
-    doNothing().when(coordinator0).shutdown(any());
-    manager0.update(EndOfStreamManager.buildEndOfStreamEnvelope(ssp1), coordinator0);
-    //verify task count is 1
-    ArgumentCaptor<Integer> argument = ArgumentCaptor.forClass(Integer.class);
-    verify(manager0).sendEndOfStream(any(), argument.capture());
-    assertTrue(argument.getValue() == 1);
-    ArgumentCaptor<TaskCoordinator.RequestScope> arg = ArgumentCaptor.forClass(TaskCoordinator.RequestScope.class);
-    verify(coordinator0).shutdown(arg.capture());
-    assertEquals(TaskCoordinator.RequestScope.CURRENT_TASK, arg.getValue());
-
-    // int1 end-of-stream
-    IncomingMessageEnvelope intEos = new IncomingMessageEnvelope(ssp2, null, null, new EndOfStreamMessage(t0.getTaskName(), 1));
-    manager1.update(intEos, coordinator1);
-    doNothing().when(coordinator1).shutdown(any());
-    verify(manager1, never()).sendEndOfStream(any(), anyInt());
-    arg = ArgumentCaptor.forClass(TaskCoordinator.RequestScope.class);
-    verify(coordinator1).shutdown(arg.capture());
-    assertEquals(TaskCoordinator.RequestScope.CURRENT_TASK, arg.getValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/test/java/org/apache/samza/control/TestIOGraph.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/control/TestIOGraph.java b/samza-core/src/test/java/org/apache/samza/control/TestIOGraph.java
deleted file mode 100644
index 39c56c3..0000000
--- a/samza-core/src/test/java/org/apache/samza/control/TestIOGraph.java
+++ /dev/null
@@ -1,200 +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.control;
-
-import java.time.Duration;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-import org.apache.samza.config.Config;
-import org.apache.samza.config.JobConfig;
-import org.apache.samza.config.MapConfig;
-import org.apache.samza.control.IOGraph.IONode;
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.OutputStream;
-import org.apache.samza.operators.StreamGraphImpl;
-import org.apache.samza.operators.functions.JoinFunction;
-import org.apache.samza.runtime.ApplicationRunner;
-import org.apache.samza.system.StreamSpec;
-import org.junit.Before;
-import org.junit.Test;
-
-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 TestIOGraph {
-  StreamSpec input1;
-  StreamSpec input2;
-  StreamSpec input3;
-  StreamSpec output1;
-  StreamSpec output2;
-  StreamSpec int1;
-  StreamSpec int2;
-
-  StreamGraphImpl streamGraph;
-
-  @Before
-  public void setup() {
-    ApplicationRunner runner = mock(ApplicationRunner.class);
-    Map<String, String> configMap = new HashMap<>();
-    configMap.put(JobConfig.JOB_NAME(), "test-app");
-    configMap.put(JobConfig.JOB_DEFAULT_SYSTEM(), "test-system");
-    Config config = new MapConfig(configMap);
-
-    /**
-     * the graph looks like the following. number of partitions in parentheses. quotes indicate expected value.
-     *
-     *                                    input1 -> map -> join -> output1
-     *                                                       |
-     *                      input2 -> partitionBy -> filter -|
-     *                                                       |
-     *           input3 -> filter -> partitionBy -> map -> join -> output2
-     *
-     */
-    input1 = new StreamSpec("input1", "input1", "system1");
-    input2 = new StreamSpec("input2", "input2", "system2");
-    input3 = new StreamSpec("input3", "input3", "system2");
-
-    output1 = new StreamSpec("output1", "output1", "system1");
-    output2 = new StreamSpec("output2", "output2", "system2");
-
-    runner = mock(ApplicationRunner.class);
-    when(runner.getStreamSpec("input1")).thenReturn(input1);
-    when(runner.getStreamSpec("input2")).thenReturn(input2);
-    when(runner.getStreamSpec("input3")).thenReturn(input3);
-    when(runner.getStreamSpec("output1")).thenReturn(output1);
-    when(runner.getStreamSpec("output2")).thenReturn(output2);
-
-    // intermediate streams used in tests
-    int1 = new StreamSpec("test-app-1-partition_by-3", "test-app-1-partition_by-3", "default-system");
-    int2 = new StreamSpec("test-app-1-partition_by-8", "test-app-1-partition_by-8", "default-system");
-    when(runner.getStreamSpec("test-app-1-partition_by-3"))
-        .thenReturn(int1);
-    when(runner.getStreamSpec("test-app-1-partition_by-8"))
-        .thenReturn(int2);
-
-    streamGraph = new StreamGraphImpl(runner, config);
-    BiFunction msgBuilder = mock(BiFunction.class);
-    MessageStream m1 = streamGraph.getInputStream("input1", msgBuilder).map(m -> m);
-    MessageStream m2 = streamGraph.getInputStream("input2", msgBuilder).partitionBy(m -> "haha").filter(m -> true);
-    MessageStream m3 = streamGraph.getInputStream("input3", msgBuilder).filter(m -> true).partitionBy(m -> "hehe").map(m -> m);
-    Function mockFn = mock(Function.class);
-    OutputStream<Object, Object, Object> om1 = streamGraph.getOutputStream("output1", mockFn, mockFn);
-    OutputStream<Object, Object, Object> om2 = streamGraph.getOutputStream("output2", mockFn, mockFn);
-
-    m1.join(m2, mock(JoinFunction.class), Duration.ofHours(2)).sendTo(om1);
-    m3.join(m2, mock(JoinFunction.class), Duration.ofHours(1)).sendTo(om2);
-  }
-
-  @Test
-  public void testBuildIOGraph() {
-    IOGraph ioGraph = streamGraph.toIOGraph();
-    assertEquals(ioGraph.getNodes().size(), 4);
-
-    for (IONode node : ioGraph.getNodes()) {
-      if (node.getOutput().equals(output1)) {
-        assertEquals(node.getInputs().size(), 2);
-        assertFalse(node.isOutputIntermediate());
-        StreamSpec[] inputs = sort(node.getInputs());
-        assertEquals(inputs[0], input1);
-        assertEquals(inputs[1], int1);
-      } else if (node.getOutput().equals(output2)) {
-        assertEquals(node.getInputs().size(), 2);
-        assertFalse(node.isOutputIntermediate());
-        StreamSpec[] inputs = sort(node.getInputs());
-        assertEquals(inputs[0], int1);
-        assertEquals(inputs[1], int2);
-      } else if (node.getOutput().equals(int1)) {
-        assertEquals(node.getInputs().size(), 1);
-        assertTrue(node.isOutputIntermediate());
-        StreamSpec[] inputs = sort(node.getInputs());
-        assertEquals(inputs[0], input2);
-      } else if (node.getOutput().equals(int2)) {
-        assertEquals(node.getInputs().size(), 1);
-        assertTrue(node.isOutputIntermediate());
-        StreamSpec[] inputs = sort(node.getInputs());
-        assertEquals(inputs[0], input3);
-      }
-    }
-  }
-
-  @Test
-  public void testNodesOfInput() {
-    IOGraph ioGraph = streamGraph.toIOGraph();
-    Collection<IONode> nodes = ioGraph.getNodesOfInput(input1.toSystemStream());
-    assertEquals(nodes.size(), 1);
-    IONode node = nodes.iterator().next();
-    assertEquals(node.getOutput(), output1);
-    assertEquals(node.getInputs().size(), 2);
-    assertFalse(node.isOutputIntermediate());
-
-    nodes = ioGraph.getNodesOfInput(input2.toSystemStream());
-    assertEquals(nodes.size(), 1);
-    node = nodes.iterator().next();
-    assertEquals(node.getOutput(), int1);
-    assertEquals(node.getInputs().size(), 1);
-    assertTrue(node.isOutputIntermediate());
-
-    nodes = ioGraph.getNodesOfInput(int1.toSystemStream());
-    assertEquals(nodes.size(), 2);
-    nodes.forEach(n -> {
-        assertEquals(n.getInputs().size(), 2);
-      });
-
-    nodes = ioGraph.getNodesOfInput(input3.toSystemStream());
-    assertEquals(nodes.size(), 1);
-    node = nodes.iterator().next();
-    assertEquals(node.getOutput(), int2);
-    assertEquals(node.getInputs().size(), 1);
-    assertTrue(node.isOutputIntermediate());
-
-    nodes = ioGraph.getNodesOfInput(int2.toSystemStream());
-    assertEquals(nodes.size(), 1);
-    node = nodes.iterator().next();
-    assertEquals(node.getOutput(), output2);
-    assertEquals(node.getInputs().size(), 2);
-    assertFalse(node.isOutputIntermediate());
-  }
-
-  private static StreamSpec[] sort(Set<StreamSpec> specs) {
-    StreamSpec[] array = new StreamSpec[specs.size()];
-    specs.toArray(array);
-    Arrays.sort(array, (s1, s2) -> s1.getId().compareTo(s2.getId()));
-    return array;
-  }
-
-  public static IOGraph buildSimpleIOGraph(List<StreamSpec> inputs,
-      StreamSpec output,
-      boolean isOutputIntermediate) {
-    IONode node = new IONode(output, isOutputIntermediate);
-    inputs.forEach(input -> node.addInput(input));
-    return new IOGraph(Collections.singleton(node));
-  }
-}


[14/16] samza git commit: SAMZA-1434: Fix issues found in Hadoop

Posted by xi...@apache.org.
SAMZA-1434: Fix issues found in Hadoop

Fix the following bugs found when running Samza on hadoop:

1. Hdfs allows output partitions to be 0 (empty folder)
2. Add null check for the changelog topic generation
3. Call getStreamSpec() instead of using streamSpec member in StreamEdge. This is due to getStreamSpec will do more transformation.
4. Bound the auto-generated intermediate topic partition by a certain count (256).

Author: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>

Reviewers: Jagadish Venkatraman <ja...@apache.org>

Closes #307 from xinyuiscool/SAMZA-1434


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

Branch: refs/heads/master
Commit: a1f01444ec12f49684213cc69b1cce16ff0f8232
Parents: 2819cbc
Author: Xinyu Liu <xi...@gmail.com>
Authored: Fri Sep 29 15:05:55 2017 -0700
Committer: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>
Committed: Fri Sep 29 15:05:55 2017 -0700

----------------------------------------------------------------------
 samza-api/src/main/java/org/apache/samza/system/StreamSpec.java | 5 +++--
 .../main/java/org/apache/samza/config/JavaStorageConfig.java    | 4 +++-
 .../main/java/org/apache/samza/execution/ExecutionPlanner.java  | 5 ++++-
 .../src/main/java/org/apache/samza/execution/StreamEdge.java    | 3 ++-
 4 files changed, 12 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/a1f01444/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java b/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
index 8d7401a..6ea1a22 100644
--- a/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
+++ b/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
@@ -158,8 +158,9 @@ public class StreamSpec {
     validateLogicalIdentifier("streamId", id);
     validateLogicalIdentifier("systemName", systemName);
 
-    if (partitionCount < 1) {
-      throw new IllegalArgumentException("Parameter 'partitionCount' must be greater than 0");
+    // partition count being 0 is a valid use case in Hadoop when the output stream is an empty folder
+    if (partitionCount < 0) {
+      throw new IllegalArgumentException("Parameter 'partitionCount' must be >= 0");
     }
 
     this.id = id;

http://git-wip-us.apache.org/repos/asf/samza/blob/a1f01444/samza-core/src/main/java/org/apache/samza/config/JavaStorageConfig.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/config/JavaStorageConfig.java b/samza-core/src/main/java/org/apache/samza/config/JavaStorageConfig.java
index 4e9a58a..34e5683 100644
--- a/samza-core/src/main/java/org/apache/samza/config/JavaStorageConfig.java
+++ b/samza-core/src/main/java/org/apache/samza/config/JavaStorageConfig.java
@@ -73,7 +73,9 @@ public class JavaStorageConfig extends MapConfig {
       systemStreamRes = systemStream;
     }
 
-    systemStreamRes = StreamManager.createUniqueNameForBatch(systemStreamRes, this);
+    if (systemStreamRes != null) {
+      systemStreamRes = StreamManager.createUniqueNameForBatch(systemStreamRes, this);
+    }
     return systemStreamRes;
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/a1f01444/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java
index e258d13..998ea1e 100644
--- a/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java
+++ b/samza-core/src/main/java/org/apache/samza/execution/ExecutionPlanner.java
@@ -50,6 +50,8 @@ import org.slf4j.LoggerFactory;
 public class ExecutionPlanner {
   private static final Logger log = LoggerFactory.getLogger(ExecutionPlanner.class);
 
+  private static final int MAX_INFERRED_PARTITIONS = 256;
+
   private final Config config;
   private final StreamManager streamManager;
 
@@ -253,9 +255,10 @@ public class ExecutionPlanner {
     if (partitions < 0) {
       // use the following simple algo to figure out the partitions
       // partition = MAX(MAX(Input topic partitions), MAX(Output topic partitions))
+      // partition will be further bounded by MAX_INFERRED_PARTITIONS. This is important when running in hadoop.
       int maxInPartitions = maxPartition(jobGraph.getSources());
       int maxOutPartitions = maxPartition(jobGraph.getSinks());
-      partitions = Math.max(maxInPartitions, maxOutPartitions);
+      partitions = Math.min(Math.max(maxInPartitions, maxOutPartitions), MAX_INFERRED_PARTITIONS);
     }
     for (StreamEdge edge : jobGraph.getIntermediateStreamEdges()) {
       if (edge.getPartitionCount() <= 0) {

http://git-wip-us.apache.org/repos/asf/samza/blob/a1f01444/samza-core/src/main/java/org/apache/samza/execution/StreamEdge.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/execution/StreamEdge.java b/samza-core/src/main/java/org/apache/samza/execution/StreamEdge.java
index f545490..792fde5 100644
--- a/samza-core/src/main/java/org/apache/samza/execution/StreamEdge.java
+++ b/samza-core/src/main/java/org/apache/samza/execution/StreamEdge.java
@@ -82,7 +82,8 @@ public class StreamEdge {
   }
 
   SystemStream getSystemStream() {
-    return new SystemStream(streamSpec.getSystemName(), streamSpec.getPhysicalName());
+    StreamSpec spec = getStreamSpec();
+    return new SystemStream(spec.getSystemName(), spec.getPhysicalName());
   }
 
   String getFormattedSystemStream() {


[02/16] samza git commit: SAMZA-1321: Propagate end-of-stream and watermark messages

Posted by xi...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/bb3007d6/samza-test/src/test/java/org/apache/samza/test/util/TestStreamConsumer.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/util/TestStreamConsumer.java b/samza-test/src/test/java/org/apache/samza/test/util/TestStreamConsumer.java
new file mode 100644
index 0000000..31eee15
--- /dev/null
+++ b/samza-test/src/test/java/org/apache/samza/test/util/TestStreamConsumer.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.test.util;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.SystemConsumer;
+import org.apache.samza.system.SystemStreamPartition;
+
+public class TestStreamConsumer implements SystemConsumer {
+  private List<IncomingMessageEnvelope> envelopes;
+
+  public TestStreamConsumer(List<IncomingMessageEnvelope> envelopes) {
+    this.envelopes = envelopes;
+  }
+
+  @Override
+  public void start() { }
+
+  @Override
+  public void stop() { }
+
+  @Override
+  public void register(SystemStreamPartition systemStreamPartition, String offset) { }
+
+  @Override
+  public Map<SystemStreamPartition, List<IncomingMessageEnvelope>> poll(
+      Set<SystemStreamPartition> systemStreamPartitions, long timeout)
+      throws InterruptedException {
+    return systemStreamPartitions.stream().collect(Collectors.toMap(ssp -> ssp, ssp -> envelopes));
+  }
+}


[13/16] samza git commit: SAMZA-1386: Inline End-of-stream and Watermark logic inside OperatorImpl

Posted by xi...@apache.org.
SAMZA-1386: Inline End-of-stream and Watermark logic inside OperatorImpl

This patch contains the following changes:
1. Refactor watermark and end-of-stream logic. The aggregation/handling has been moved from WatermarkManager/EndOfStreamManager to be inline inside OperatorImpl. This is for keeping the logic in one place.
2. Now subclass of OperatorImpl will override handleWatermark() to do its specific handling, such as fire trigger.
3. Add emitWatermark() in OperatorImpl so subclass can call it to emit watermark upon receiving a message or watermark.

Author: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>
Author: Xinyu Liu <xi...@xiliu-ld.linkedin.biz>

Reviewers: Yi Pan <ni...@gmail.com>

Closes #277 from xinyuiscool/SAMZA-1386


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

Branch: refs/heads/master
Commit: 2819cbc7691b569b3eef66d702746d9e34b3e745
Parents: 4754148
Author: Xinyu Liu <xi...@gmail.com>
Authored: Thu Sep 28 17:11:28 2017 -0700
Committer: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>
Committed: Thu Sep 28 17:11:28 2017 -0700

----------------------------------------------------------------------
 build.gradle                                    |   1 +
 .../operators/functions/WatermarkFunction.java  |  58 ++++
 .../org/apache/samza/system/ControlMessage.java |  46 +++
 .../apache/samza/system/EndOfStreamMessage.java |  38 +++
 .../samza/system/IncomingMessageEnvelope.java   |   7 +-
 .../org/apache/samza/system/MessageType.java    |  45 +++
 .../apache/samza/system/WatermarkMessage.java   |  46 +++
 .../apache/samza/container/TaskContextImpl.java | 131 ++++++++
 .../control/ControlMessageListenerTask.java     |  49 ---
 .../samza/control/ControlMessageUtils.java      |  81 -----
 .../samza/control/EndOfStreamManager.java       | 159 ---------
 .../java/org/apache/samza/control/IOGraph.java  | 113 -------
 .../org/apache/samza/control/Watermark.java     |  57 ----
 .../apache/samza/control/WatermarkManager.java  | 187 -----------
 .../apache/samza/message/ControlMessage.java    |  52 ---
 .../samza/message/EndOfStreamMessage.java       |  36 --
 .../samza/message/IntermediateMessageType.java  |  46 ---
 .../org/apache/samza/message/MessageType.java   |  46 ---
 .../apache/samza/message/WatermarkMessage.java  |  43 ---
 .../apache/samza/operators/StreamGraphImpl.java |   5 -
 .../operators/impl/ControlMessageSender.java    |  56 ++++
 .../samza/operators/impl/EndOfStreamStates.java |  98 ++++++
 .../samza/operators/impl/OperatorImpl.java      | 198 ++++++++---
 .../samza/operators/impl/OperatorImplGraph.java |  98 +++++-
 .../operators/impl/OutputOperatorImpl.java      |  32 +-
 .../samza/operators/impl/WatermarkStates.java   | 119 +++++++
 .../samza/operators/spec/InputOperatorSpec.java |   6 +
 .../samza/operators/spec/JoinOperatorSpec.java  |   6 +
 .../samza/operators/spec/OperatorSpec.java      |   3 +
 .../operators/spec/OutputOperatorSpec.java      |   7 +
 .../samza/operators/spec/SinkOperatorSpec.java  |   6 +
 .../operators/spec/StreamOperatorSpec.java      |   6 +
 .../operators/spec/WindowOperatorSpec.java      |   8 +
 .../serializers/IntermediateMessageSerde.java   |  18 +-
 .../samza/task/AsyncStreamTaskAdapter.java      |  22 +-
 .../apache/samza/task/StreamOperatorTask.java   |  45 ++-
 .../apache/samza/checkpoint/OffsetManager.scala |   7 +-
 .../apache/samza/container/TaskInstance.scala   | 185 ++---------
 .../apache/samza/serializers/SerdeManager.scala |   5 +-
 .../samza/control/TestControlMessageUtils.java  | 115 -------
 .../samza/control/TestEndOfStreamManager.java   | 333 -------------------
 .../org/apache/samza/control/TestIOGraph.java   | 200 -----------
 .../samza/control/TestWatermarkManager.java     | 260 ---------------
 .../samza/operators/TestJoinOperator.java       |   3 +-
 .../samza/operators/TestWindowOperator.java     |   6 +-
 .../impl/TestControlMessageSender.java          |  73 ++++
 .../operators/impl/TestEndOfStreamStates.java   |  78 +++++
 .../samza/operators/impl/TestOperatorImpl.java  |  32 +-
 .../operators/impl/TestOperatorImplGraph.java   | 181 +++++++++-
 .../operators/impl/TestWatermarkStates.java     | 102 ++++++
 .../TestIntermediateMessageSerde.java           |  18 +-
 .../org/apache/samza/task/TestAsyncRunLoop.java |   7 +-
 .../samza/container/TestTaskInstance.scala      |  63 +---
 .../samza/serializers/TestSerdeManager.scala    |  10 +-
 .../samza/system/hdfs/HdfsSystemConsumer.java   |   3 +-
 .../EndOfStreamIntegrationTest.java             |   2 +-
 .../WatermarkIntegrationTest.java               |  14 +-
 .../samza/test/util/ArraySystemConsumer.java    |   3 +-
 58 files changed, 1521 insertions(+), 2153 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 319e51f..e10b970 100644
--- a/build.gradle
+++ b/build.gradle
@@ -125,6 +125,7 @@ project(':samza-api') {
 
   dependencies {
     compile "org.slf4j:slf4j-api:$slf4jVersion"
+    compile "org.codehaus.jackson:jackson-mapper-asl:$jacksonVersion"
     testCompile "junit:junit:$junitVersion"
     testCompile "org.mockito:mockito-all:$mockitoVersion"
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-api/src/main/java/org/apache/samza/operators/functions/WatermarkFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/WatermarkFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/WatermarkFunction.java
new file mode 100644
index 0000000..9c4b9bf
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/WatermarkFunction.java
@@ -0,0 +1,58 @@
+/*
+ * 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;
+
+/**
+ * Allows user-specific handling of Watermark
+ */
+public interface WatermarkFunction {
+
+  /**
+   * Processes the input watermark coming from upstream operators.
+   * This allows user-defined watermark handling, such as trigger events
+   * or propagate it to downstream.
+   * @param watermark input watermark
+   */
+  void processWatermark(long watermark);
+
+  /**
+   * Returns the output watermark. This function will be invoked immediately after either
+   * of the following events:
+   *
+   * <ol>
+   *
+   * <li> Return of the transform function, e.g. {@link FlatMapFunction}.
+   *
+   * <li> Return of the processWatermark function.
+   *
+   * </ol>
+   *
+   *
+   *
+   * Note: If the transform function returns a collection of output, the output watermark
+   * will be emitted after the output collection is propagated to downstream operators. So
+   * it might delay the watermark propagation. The delay will cause more buffering and might
+   * have performance impact.
+   *
+   * @return output watermark, or null if the output watermark should not be updated. Samza
+   * guarantees that the same watermark value will be only emitted once.
+   */
+  Long getOutputWatermark();
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-api/src/main/java/org/apache/samza/system/ControlMessage.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/ControlMessage.java b/samza-api/src/main/java/org/apache/samza/system/ControlMessage.java
new file mode 100644
index 0000000..4ec58b4
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/ControlMessage.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.system;
+
+/**
+ * The abstract class of all control messages, containing
+ * the task that produces the control message, the total number of producer tasks,
+ * and a version number.
+ */
+public abstract class ControlMessage {
+  private final String taskName;
+  private int version = 1;
+
+  public ControlMessage(String taskName) {
+    this.taskName = taskName;
+  }
+
+  public String getTaskName() {
+    return taskName;
+  }
+
+  public void setVersion(int version) {
+    this.version = version;
+  }
+
+  public int getVersion() {
+    return version;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-api/src/main/java/org/apache/samza/system/EndOfStreamMessage.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/EndOfStreamMessage.java b/samza-api/src/main/java/org/apache/samza/system/EndOfStreamMessage.java
new file mode 100644
index 0000000..59d0356
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/EndOfStreamMessage.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.system;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ *  The EndOfStreamMessage is a control message that is sent out to next stage
+ *  once the task has consumed to the end of a bounded stream.
+ */
+public class EndOfStreamMessage extends ControlMessage {
+  public EndOfStreamMessage() {
+    this(null);
+  }
+
+  @JsonCreator
+  public EndOfStreamMessage(@JsonProperty("task-name") String taskName) {
+    super(taskName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-api/src/main/java/org/apache/samza/system/IncomingMessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/IncomingMessageEnvelope.java b/samza-api/src/main/java/org/apache/samza/system/IncomingMessageEnvelope.java
index 9182522..96fa81c 100644
--- a/samza-api/src/main/java/org/apache/samza/system/IncomingMessageEnvelope.java
+++ b/samza-api/src/main/java/org/apache/samza/system/IncomingMessageEnvelope.java
@@ -96,9 +96,12 @@ public class IncomingMessageEnvelope {
    * @param ssp The SSP that is at end-of-stream.
    * @return an IncomingMessageEnvelope corresponding to end-of-stream for that SSP.
    */
-  @Deprecated
   public static IncomingMessageEnvelope buildEndOfStreamEnvelope(SystemStreamPartition ssp) {
-    return new IncomingMessageEnvelope(ssp, END_OF_STREAM_OFFSET, null, null);
+    return new IncomingMessageEnvelope(ssp, END_OF_STREAM_OFFSET, null, new EndOfStreamMessage(null));
+  }
+
+  public static IncomingMessageEnvelope buildWatermarkEnvelope(SystemStreamPartition ssp, long watermark) {
+    return new IncomingMessageEnvelope(ssp, null, null, new WatermarkMessage(watermark, null));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-api/src/main/java/org/apache/samza/system/MessageType.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/MessageType.java b/samza-api/src/main/java/org/apache/samza/system/MessageType.java
new file mode 100644
index 0000000..7129d00
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/MessageType.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.system;
+
+/**
+ * The type of the intermediate stream message. The enum will be encoded using its ordinal value and
+ * put in the first byte of the serialization of intermediate message.
+ */
+public enum MessageType {
+  USER_MESSAGE,
+  WATERMARK,
+  END_OF_STREAM;
+
+  /**
+   * Returns the {@link MessageType} of a particular intermediate stream message.
+   * @param message an intermediate stream message
+   * @return type of the message
+   */
+  public static MessageType of(Object message) {
+    if (message instanceof WatermarkMessage) {
+      return WATERMARK;
+    } else if (message instanceof EndOfStreamMessage) {
+      return END_OF_STREAM;
+    } else {
+      return USER_MESSAGE;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-api/src/main/java/org/apache/samza/system/WatermarkMessage.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/WatermarkMessage.java b/samza-api/src/main/java/org/apache/samza/system/WatermarkMessage.java
new file mode 100644
index 0000000..7278c5c
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/WatermarkMessage.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.system;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ *  The WatermarkMessage is a control message that is sent out to next stage
+ *  with a watermark timestamp and the task that produces the watermark.
+ */
+public class WatermarkMessage extends ControlMessage {
+  private final long timestamp;
+
+  public WatermarkMessage(long watermark) {
+    this(watermark, null);
+  }
+
+  @JsonCreator
+  public WatermarkMessage(@JsonProperty("timestamp") long timestamp,
+                          @JsonProperty("task-name") String taskName) {
+    super(taskName);
+    this.timestamp = timestamp;
+  }
+
+  public long getTimestamp() {
+    return timestamp;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/container/TaskContextImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/container/TaskContextImpl.java b/samza-core/src/main/java/org/apache/samza/container/TaskContextImpl.java
new file mode 100644
index 0000000..7990d2b
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/container/TaskContextImpl.java
@@ -0,0 +1,131 @@
+/*
+ * 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.container;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.samza.checkpoint.OffsetManager;
+import org.apache.samza.job.model.JobModel;
+import org.apache.samza.metrics.ReadableMetricsRegistry;
+import org.apache.samza.storage.StorageEngine;
+import org.apache.samza.storage.TaskStorageManager;
+import org.apache.samza.system.StreamMetadataCache;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.task.TaskContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+public class TaskContextImpl implements TaskContext {
+  private static final Logger LOG = LoggerFactory.getLogger(TaskContextImpl.class);
+
+  private final TaskName taskName;
+  private final TaskInstanceMetrics metrics;
+  private final SamzaContainerContext containerContext;
+  private final Set<SystemStreamPartition> systemStreamPartitions;
+  private final OffsetManager offsetManager;
+  private final TaskStorageManager storageManager;
+  private final JobModel jobModel;
+  private final StreamMetadataCache streamMetadataCache;
+  private final Map<String, Object> objectRegistry = new HashMap<>();
+
+  private Object userContext = null;
+
+  public TaskContextImpl(TaskName taskName,
+                         TaskInstanceMetrics metrics,
+                         SamzaContainerContext containerContext,
+                         Set<SystemStreamPartition> systemStreamPartitions,
+                         OffsetManager offsetManager,
+                         TaskStorageManager storageManager,
+                         JobModel jobModel,
+                         StreamMetadataCache streamMetadataCache) {
+    this.taskName = taskName;
+    this.metrics = metrics;
+    this.containerContext = containerContext;
+    this.systemStreamPartitions = ImmutableSet.copyOf(systemStreamPartitions);
+    this.offsetManager = offsetManager;
+    this.storageManager = storageManager;
+    this.jobModel = jobModel;
+    this.streamMetadataCache = streamMetadataCache;
+  }
+
+  @Override
+  public ReadableMetricsRegistry getMetricsRegistry() {
+    return metrics.registry();
+  }
+
+  @Override
+  public Set<SystemStreamPartition> getSystemStreamPartitions() {
+    return systemStreamPartitions;
+  }
+
+  @Override
+  public StorageEngine getStore(String storeName) {
+    if (storageManager != null) {
+      return storageManager.apply(storeName);
+    } else {
+      LOG.warn("No store found for name: {}", storeName);
+      return null;
+    }
+  }
+
+  @Override
+  public TaskName getTaskName() {
+    return taskName;
+  }
+
+  @Override
+  public SamzaContainerContext getSamzaContainerContext() {
+    return containerContext;
+  }
+
+  @Override
+  public void setStartingOffset(SystemStreamPartition ssp, String offset) {
+    offsetManager.setStartingOffset(taskName, ssp, offset);
+  }
+
+  @Override
+  public void setUserContext(Object context) {
+    userContext = context;
+  }
+
+  @Override
+  public Object getUserContext() {
+    return userContext;
+  }
+
+  public void registerObject(String name, Object value) {
+    objectRegistry.put(name, value);
+  }
+
+  public Object fetchObject(String name) {
+    return objectRegistry.get(name);
+  }
+
+  public JobModel getJobModel() {
+    return jobModel;
+  }
+
+  public StreamMetadataCache getStreamMetadataCache() {
+    return streamMetadataCache;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/control/ControlMessageListenerTask.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/control/ControlMessageListenerTask.java b/samza-core/src/main/java/org/apache/samza/control/ControlMessageListenerTask.java
deleted file mode 100644
index 9e4b40a..0000000
--- a/samza-core/src/main/java/org/apache/samza/control/ControlMessageListenerTask.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.control;
-
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-
-
-/**
- * The listener interface for the aggregation result of control messages.
- * Any task that handles control messages such as {@link org.apache.samza.message.EndOfStreamMessage}
- * and {@link org.apache.samza.message.WatermarkMessage} needs to implement this interface.
- */
-public interface ControlMessageListenerTask {
-
-  /**
-   * Returns the topology of the streams. Any control message listener needs to
-   * provide this topology so Samza can propagate the control message to downstreams.
-   * @return {@link IOGraph} of input to output streams. It
-   */
-  IOGraph getIOGraph();
-
-  /**
-   * Invoked when a Watermark comes.
-   * @param watermark contains the watermark timestamp
-   * @param systemStream source of stream that emits the watermark
-   * @param collector message collector
-   * @param coordinator task coordinator
-   */
-  void onWatermark(Watermark watermark, SystemStream systemStream, MessageCollector collector, TaskCoordinator coordinator);
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/control/ControlMessageUtils.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/control/ControlMessageUtils.java b/samza-core/src/main/java/org/apache/samza/control/ControlMessageUtils.java
deleted file mode 100644
index ebb0d86..0000000
--- a/samza-core/src/main/java/org/apache/samza/control/ControlMessageUtils.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.control;
-
-import com.google.common.collect.Multimap;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.stream.Collectors;
-import org.apache.samza.message.ControlMessage;
-import org.apache.samza.system.OutgoingMessageEnvelope;
-import org.apache.samza.system.StreamMetadataCache;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.system.SystemStreamMetadata;
-import org.apache.samza.task.MessageCollector;
-
-
-/**
- * This class privates static utils for handling control messages
- */
-class ControlMessageUtils {
-
-  /**
-   * Send a control message to every partition of the {@link SystemStream}
-   * @param message control message
-   * @param systemStream the stream to sent
-   * @param metadataCache stream metadata cache
-   * @param collector collector to send the message
-   */
-  static void sendControlMessage(ControlMessage message,
-      SystemStream systemStream,
-      StreamMetadataCache metadataCache,
-      MessageCollector collector) {
-    SystemStreamMetadata metadata = metadataCache.getSystemStreamMetadata(systemStream, true);
-    int partitionCount = metadata.getSystemStreamPartitionMetadata().size();
-    for (int i = 0; i < partitionCount; i++) {
-      OutgoingMessageEnvelope envelopeOut = new OutgoingMessageEnvelope(systemStream, i, "", message);
-      collector.send(envelopeOut);
-    }
-  }
-
-  /**
-   * Calculate the mapping from an output stream to the number of upstream tasks that will produce to the output stream
-   * @param inputToTasks input stream to its consumer tasks mapping
-   * @param ioGraph topology of the stream inputs and outputs
-   * @return mapping from output to upstream task count
-   */
-  static Map<SystemStream, Integer> calculateUpstreamTaskCounts(Multimap<SystemStream, String> inputToTasks,
-      IOGraph ioGraph) {
-    if (ioGraph == null) {
-      return Collections.EMPTY_MAP;
-    }
-    Map<SystemStream, Integer> outputTaskCount = new HashMap<>();
-    ioGraph.getNodes().forEach(node -> {
-        // for each input stream, find out the tasks that are consuming this input stream using the inputToTasks mapping,
-        // then count the number of tasks
-        int count = node.getInputs().stream().flatMap(spec -> inputToTasks.get(spec.toSystemStream()).stream())
-            .collect(Collectors.toSet()).size();
-        // put the count of input tasks to the result
-        outputTaskCount.put(node.getOutput().toSystemStream(), count);
-      });
-    return outputTaskCount;
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/control/EndOfStreamManager.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/control/EndOfStreamManager.java b/samza-core/src/main/java/org/apache/samza/control/EndOfStreamManager.java
deleted file mode 100644
index 78a8741..0000000
--- a/samza-core/src/main/java/org/apache/samza/control/EndOfStreamManager.java
+++ /dev/null
@@ -1,159 +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.control;
-
-import com.google.common.collect.Multimap;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import org.apache.samza.message.EndOfStreamMessage;
-import org.apache.samza.system.IncomingMessageEnvelope;
-import org.apache.samza.system.StreamMetadataCache;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.system.SystemStreamPartition;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-/**
- * This class handles the end-of-stream control message. It aggregates the end-of-stream state for each input ssps of
- * a task, and propagate the eos messages to downstream intermediate streams if needed.
- *
- * Internal use only.
- */
-public class EndOfStreamManager {
-  private static final Logger log = LoggerFactory.getLogger(EndOfStreamManager.class);
-
-  private final String taskName;
-  private final MessageCollector collector;
-  // end-of-stream state per ssp
-  private final Map<SystemStreamPartition, EndOfStreamState> eosStates;
-  private final StreamMetadataCache metadataCache;
-  // topology information. Set during init()
-  private final ControlMessageListenerTask listener;
-  // mapping from output stream to its upstream task count
-  private final Map<SystemStream, Integer> upstreamTaskCounts;
-
-  public EndOfStreamManager(String taskName,
-      ControlMessageListenerTask listener,
-      Multimap<SystemStream, String> inputToTasks,
-      Set<SystemStreamPartition> ssps,
-      StreamMetadataCache metadataCache,
-      MessageCollector collector) {
-    this.taskName = taskName;
-    this.listener = listener;
-    this.metadataCache = metadataCache;
-    this.collector = collector;
-    Map<SystemStreamPartition, EndOfStreamState> states = new HashMap<>();
-    ssps.forEach(ssp -> {
-        states.put(ssp, new EndOfStreamState());
-      });
-    this.eosStates = Collections.unmodifiableMap(states);
-    this.upstreamTaskCounts = ControlMessageUtils.calculateUpstreamTaskCounts(inputToTasks, listener.getIOGraph());
-  }
-
-  public void update(IncomingMessageEnvelope envelope, TaskCoordinator coordinator) {
-    EndOfStreamState state = eosStates.get(envelope.getSystemStreamPartition());
-    EndOfStreamMessage message = (EndOfStreamMessage) envelope.getMessage();
-    state.update(message.getTaskName(), message.getTaskCount());
-    log.info("Received end-of-stream from task " + message.getTaskName() + " in " + envelope.getSystemStreamPartition());
-
-    SystemStream systemStream = envelope.getSystemStreamPartition().getSystemStream();
-    if (isEndOfStream(systemStream)) {
-      log.info("End-of-stream of input " + systemStream + " for " + systemStream);
-      listener.getIOGraph().getNodesOfInput(systemStream).forEach(node -> {
-          // find the intermediate streams that need broadcast the eos messages
-          if (node.isOutputIntermediate()) {
-            // check all the input stream partitions assigned to the task are end-of-stream
-            boolean inputsEndOfStream = node.getInputs().stream().allMatch(spec -> isEndOfStream(spec.toSystemStream()));
-            if (inputsEndOfStream) {
-              // broadcast the end-of-stream message to the intermediate stream
-              SystemStream outputStream = node.getOutput().toSystemStream();
-              sendEndOfStream(outputStream, upstreamTaskCounts.get(outputStream));
-            }
-          }
-        });
-
-      boolean allEndOfStream = eosStates.values().stream().allMatch(EndOfStreamState::isEndOfStream);
-      if (allEndOfStream) {
-        // all inputs have been end-of-stream, shut down the task
-        log.info("All input streams have reached the end for task " + taskName);
-        coordinator.shutdown(TaskCoordinator.RequestScope.CURRENT_TASK);
-      }
-    }
-  }
-
-  /**
-   * Return true if all partitions of the systemStream that are assigned to the current task have reached EndOfStream.
-   * @param systemStream stream
-   * @return whether the stream reaches to the end for this task
-   */
-  boolean isEndOfStream(SystemStream systemStream) {
-    return eosStates.entrySet().stream()
-        .filter(entry -> entry.getKey().getSystemStream().equals(systemStream))
-        .allMatch(entry -> entry.getValue().isEndOfStream());
-  }
-
-  /**
-   * Send the EndOfStream control messages to downstream
-   * @param systemStream downstream stream
-   */
-  void sendEndOfStream(SystemStream systemStream, int taskCount) {
-    log.info("Send end-of-stream messages with upstream task count {} to all partitions of {}", taskCount, systemStream);
-    final EndOfStreamMessage message = new EndOfStreamMessage(taskName, taskCount);
-    ControlMessageUtils.sendControlMessage(message, systemStream, metadataCache, collector);
-  }
-
-  /**
-   * This class keeps the internal state for a ssp to be end-of-stream.
-   */
-  final static class EndOfStreamState {
-    // set of upstream tasks
-    private final Set<String> tasks = new HashSet<>();
-    private int expectedTotal = Integer.MAX_VALUE;
-    private boolean isEndOfStream = false;
-
-    void update(String taskName, int taskCount) {
-      if (taskName != null) {
-        tasks.add(taskName);
-      }
-      expectedTotal = taskCount;
-      isEndOfStream = tasks.size() == expectedTotal;
-    }
-
-    boolean isEndOfStream() {
-      return isEndOfStream;
-    }
-  }
-
-  /**
-   * Build an end-of-stream envelope for an ssp of a source input.
-   *
-   * @param ssp The SSP that is at end-of-stream.
-   * @return an IncomingMessageEnvelope corresponding to end-of-stream for that SSP.
-   */
-  public static IncomingMessageEnvelope buildEndOfStreamEnvelope(SystemStreamPartition ssp) {
-    return new IncomingMessageEnvelope(ssp, IncomingMessageEnvelope.END_OF_STREAM_OFFSET, null, new EndOfStreamMessage(null, 0));
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/control/IOGraph.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/control/IOGraph.java b/samza-core/src/main/java/org/apache/samza/control/IOGraph.java
deleted file mode 100644
index a30c13d..0000000
--- a/samza-core/src/main/java/org/apache/samza/control/IOGraph.java
+++ /dev/null
@@ -1,113 +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.control;
-
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Multimap;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import org.apache.samza.operators.StreamGraphImpl;
-import org.apache.samza.operators.spec.OperatorSpec;
-import org.apache.samza.operators.spec.OutputOperatorSpec;
-import org.apache.samza.system.StreamSpec;
-import org.apache.samza.system.SystemStream;
-
-
-/**
- * This class provides the topology of stream inputs to outputs.
- */
-public class IOGraph {
-
-  public static final class IONode {
-    private final Set<StreamSpec> inputs = new HashSet<>();
-    private final StreamSpec output;
-    private final boolean isOutputIntermediate;
-
-    IONode(StreamSpec output, boolean isOutputIntermediate) {
-      this.output = output;
-      this.isOutputIntermediate = isOutputIntermediate;
-    }
-
-    void addInput(StreamSpec input) {
-      inputs.add(input);
-    }
-
-    public Set<StreamSpec> getInputs() {
-      return Collections.unmodifiableSet(inputs);
-    }
-
-    public StreamSpec getOutput() {
-      return output;
-    }
-
-    public boolean isOutputIntermediate() {
-      return isOutputIntermediate;
-    }
-  }
-
-  final Collection<IONode> nodes;
-  final Multimap<SystemStream, IONode> inputToNodes;
-
-  public IOGraph(Collection<IONode> nodes) {
-    this.nodes = Collections.unmodifiableCollection(nodes);
-    this.inputToNodes = HashMultimap.create();
-    nodes.forEach(node -> {
-        node.getInputs().forEach(stream -> {
-            inputToNodes.put(new SystemStream(stream.getSystemName(), stream.getPhysicalName()), node);
-          });
-      });
-  }
-
-  public Collection<IONode> getNodes() {
-    return this.nodes;
-  }
-
-  public Collection<IONode> getNodesOfInput(SystemStream input) {
-    return inputToNodes.get(input);
-  }
-
-  public static IOGraph buildIOGraph(StreamGraphImpl streamGraph) {
-    Map<Integer, IONode> nodes = new HashMap<>();
-    streamGraph.getInputOperators().entrySet().stream()
-        .forEach(entry -> buildIONodes(entry.getKey(), entry.getValue(), nodes));
-    return new IOGraph(nodes.values());
-  }
-
-  /* package private */
-  static void buildIONodes(StreamSpec input, OperatorSpec opSpec, Map<Integer, IONode> ioGraph) {
-    if (opSpec instanceof OutputOperatorSpec) {
-      OutputOperatorSpec outputOpSpec = (OutputOperatorSpec) opSpec;
-      IONode node = ioGraph.get(opSpec.getOpId());
-      if (node == null) {
-        StreamSpec output = outputOpSpec.getOutputStream().getStreamSpec();
-        node = new IONode(output, outputOpSpec.getOpCode() == OperatorSpec.OpCode.PARTITION_BY);
-        ioGraph.put(opSpec.getOpId(), node);
-      }
-      node.addInput(input);
-    }
-
-    Collection<OperatorSpec> nextOperators = opSpec.getRegisteredOperatorSpecs();
-    nextOperators.forEach(spec -> buildIONodes(input, spec, ioGraph));
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/control/Watermark.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/control/Watermark.java b/samza-core/src/main/java/org/apache/samza/control/Watermark.java
deleted file mode 100644
index a11e3b0..0000000
--- a/samza-core/src/main/java/org/apache/samza/control/Watermark.java
+++ /dev/null
@@ -1,57 +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.control;
-
-import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.system.SystemStream;
-
-
-/**
- * A watermark is a monotonically increasing value, which represents the point up to which the
- * system believes it has received all of the data before the watermark timestamp. Data that arrives
- * with a timestamp that is before the watermark is considered late.
- *
- * <p>This is the aggregate result from the WatermarkManager, which keeps track of the control message
- * {@link org.apache.samza.message.WatermarkMessage} and aggregate by returning the min of all watermark timestamp
- * in each partition.
- */
-@InterfaceStability.Unstable
-public interface Watermark {
-  /**
-   * Returns the timestamp of the watermark
-   * Note that if the task consumes more than one partitions of this stream, the watermark emitted is the min of
-   * watermarks across all partitions.
-   * @return timestamp
-   */
-  long getTimestamp();
-
-  /**
-   * Propagates the watermark to an intermediate stream
-   * @param systemStream intermediate stream
-   */
-  void propagate(SystemStream systemStream);
-
-  /**
-   * Create a copy of the watermark with the timestamp
-   * @param timestamp new timestamp
-   * @return new watermark
-   */
-  Watermark copyWithTimestamp(long timestamp);
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/control/WatermarkManager.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/control/WatermarkManager.java b/samza-core/src/main/java/org/apache/samza/control/WatermarkManager.java
deleted file mode 100644
index c4fdd88..0000000
--- a/samza-core/src/main/java/org/apache/samza/control/WatermarkManager.java
+++ /dev/null
@@ -1,187 +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.control;
-
-import com.google.common.collect.Multimap;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import org.apache.samza.message.WatermarkMessage;
-import org.apache.samza.system.IncomingMessageEnvelope;
-import org.apache.samza.system.StreamMetadataCache;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.system.SystemStreamPartition;
-import org.apache.samza.task.MessageCollector;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-/**
- * This class manages watermarks. It aggregates the watermark control messages from the upstage tasks
- * for each SSP into an envelope of {@link Watermark}, and provide a dispatcher to propagate it to downstream.
- *
- * Internal use only.
- */
-public class WatermarkManager {
-  private static final Logger log = LoggerFactory.getLogger(WatermarkManager.class);
-  public static final long TIME_NOT_EXIST = -1;
-
-  private final String taskName;
-  private final Map<SystemStreamPartition, WatermarkState> watermarkStates;
-  private final Map<SystemStream, Long> watermarkPerStream;
-  private final StreamMetadataCache metadataCache;
-  private final MessageCollector collector;
-  // mapping from output stream to its upstream task count
-  private final Map<SystemStream, Integer> upstreamTaskCounts;
-
-  public WatermarkManager(String taskName,
-      ControlMessageListenerTask listener,
-      Multimap<SystemStream, String> inputToTasks,
-      Set<SystemStreamPartition> ssps,
-      StreamMetadataCache metadataCache,
-      MessageCollector collector) {
-    this.taskName = taskName;
-    this.watermarkPerStream = new HashMap<>();
-    this.metadataCache = metadataCache;
-    this.collector = collector;
-    this.upstreamTaskCounts = ControlMessageUtils.calculateUpstreamTaskCounts(inputToTasks, listener.getIOGraph());
-
-    Map<SystemStreamPartition, WatermarkState> states = new HashMap<>();
-    ssps.forEach(ssp -> {
-        states.put(ssp, new WatermarkState());
-        watermarkPerStream.put(ssp.getSystemStream(), TIME_NOT_EXIST);
-      });
-    this.watermarkStates = Collections.unmodifiableMap(states);
-  }
-
-  /**
-   * Update the watermark based on the incoming watermark message. The message contains
-   * a timestamp and the upstream producer task. The aggregation result is the minimal value
-   * of all watermarks for the stream:
-   * <ul>
-   *   <li>Watermark(ssp) = min { Watermark(task) | task is upstream producer and the count equals total expected tasks } </li>
-   *   <li>Watermark(stream) = min { Watermark(ssp) | ssp is a partition of stream that assigns to this task } </li>
-   * </ul>
-   *
-   * @param envelope the envelope contains {@link WatermarkMessage}
-   * @return watermark envelope if there is a new aggregate watermark for the stream
-   */
-  public Watermark update(IncomingMessageEnvelope envelope) {
-    SystemStreamPartition ssp = envelope.getSystemStreamPartition();
-    WatermarkState state = watermarkStates.get(ssp);
-    WatermarkMessage message = (WatermarkMessage) envelope.getMessage();
-    state.update(message.getTimestamp(), message.getTaskName(), message.getTaskCount());
-
-    if (state.getWatermarkTime() != TIME_NOT_EXIST) {
-      long minTimestamp = watermarkStates.entrySet().stream()
-          .filter(entry -> entry.getKey().getSystemStream().equals(ssp.getSystemStream()))
-          .map(entry -> entry.getValue().getWatermarkTime())
-          .min(Long::compare)
-          .get();
-      Long curWatermark = watermarkPerStream.get(ssp.getSystemStream());
-      if (curWatermark == null || curWatermark < minTimestamp) {
-        watermarkPerStream.put(ssp.getSystemStream(), minTimestamp);
-        return new WatermarkImpl(minTimestamp);
-      }
-    }
-
-    return null;
-  }
-
-  /* package private */
-  long getWatermarkTime(SystemStreamPartition ssp) {
-    return watermarkStates.get(ssp).getWatermarkTime();
-  }
-
-  /**
-   * Send the watermark message to all partitions of an intermediate stream
-   * @param timestamp watermark timestamp
-   * @param systemStream intermediate stream
-   */
-  void sendWatermark(long timestamp, SystemStream systemStream, int taskCount) {
-    log.info("Send end-of-stream messages to all partitions of " + systemStream);
-    final WatermarkMessage watermarkMessage = new WatermarkMessage(timestamp, taskName, taskCount);
-    ControlMessageUtils.sendControlMessage(watermarkMessage, systemStream, metadataCache, collector);
-  }
-
-  /**
-   * Per ssp state of the watermarks. This class keeps track of the latest watermark timestamp
-   * from each upstream producer tasks, and use the min to update the aggregated watermark time.
-   */
-  final static class WatermarkState {
-    private int expectedTotal = Integer.MAX_VALUE;
-    private final Map<String, Long> timestamps = new HashMap<>();
-    private long watermarkTime = TIME_NOT_EXIST;
-
-    void update(long timestamp, String taskName, int taskCount) {
-      if (taskName != null) {
-        timestamps.put(taskName, timestamp);
-      }
-      expectedTotal = taskCount;
-
-      if (timestamps.size() == expectedTotal) {
-        Optional<Long> min = timestamps.values().stream().min(Long::compare);
-        watermarkTime = min.orElse(timestamp);
-      }
-    }
-
-    long getWatermarkTime() {
-      return watermarkTime;
-    }
-  }
-
-  /**
-   * Implementation of the Watermark. It keeps a reference to the {@link WatermarkManager}
-   */
-  class WatermarkImpl implements Watermark {
-    private final long timestamp;
-
-    WatermarkImpl(long timestamp) {
-      this.timestamp = timestamp;
-    }
-
-    @Override
-    public long getTimestamp() {
-      return timestamp;
-    }
-
-    @Override
-    public void propagate(SystemStream systemStream) {
-      sendWatermark(timestamp, systemStream, upstreamTaskCounts.get(systemStream));
-    }
-
-    @Override
-    public Watermark copyWithTimestamp(long time) {
-      return new WatermarkImpl(time);
-    }
-  }
-
-  /**
-   * Build a watermark control message envelope for an ssp of a source input.
-   * @param timestamp watermark time
-   * @param ssp {@link SystemStreamPartition} where the watermark coming from.
-   * @return envelope of the watermark control message
-   */
-  public static IncomingMessageEnvelope buildWatermarkEnvelope(long timestamp, SystemStreamPartition ssp) {
-    return new IncomingMessageEnvelope(ssp, null, "", new WatermarkMessage(timestamp, null, 0));
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/message/ControlMessage.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/message/ControlMessage.java b/samza-core/src/main/java/org/apache/samza/message/ControlMessage.java
deleted file mode 100644
index 46bf559..0000000
--- a/samza-core/src/main/java/org/apache/samza/message/ControlMessage.java
+++ /dev/null
@@ -1,52 +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.message;
-
-/**
- * The abstract class of all control messages, containing
- * the task that produces the control message, the total number of producer tasks,
- * and a version number.
- */
-public abstract class ControlMessage {
-  private final String taskName;
-  private final int taskCount;
-  private int version = 1;
-
-  public ControlMessage(String taskName, int taskCount) {
-    this.taskName = taskName;
-    this.taskCount = taskCount;
-  }
-
-  public String getTaskName() {
-    return taskName;
-  }
-
-  public int getTaskCount() {
-    return taskCount;
-  }
-
-  public void setVersion(int version) {
-    this.version = version;
-  }
-
-  public int getVersion() {
-    return version;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/message/EndOfStreamMessage.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/message/EndOfStreamMessage.java b/samza-core/src/main/java/org/apache/samza/message/EndOfStreamMessage.java
deleted file mode 100644
index 91981a9..0000000
--- a/samza-core/src/main/java/org/apache/samza/message/EndOfStreamMessage.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.message;
-
-import org.codehaus.jackson.annotate.JsonCreator;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/**
- *  The EndOfStreamMessage is a control message that is sent out to next stage
- *  once the task has consumed to the end of a bounded stream.
- */
-public class EndOfStreamMessage extends ControlMessage {
-
-  @JsonCreator
-  public EndOfStreamMessage(@JsonProperty("task-name") String taskName,
-                            @JsonProperty("task-count") int taskCount) {
-    super(taskName, taskCount);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/message/IntermediateMessageType.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/message/IntermediateMessageType.java b/samza-core/src/main/java/org/apache/samza/message/IntermediateMessageType.java
deleted file mode 100644
index 25fbb14..0000000
--- a/samza-core/src/main/java/org/apache/samza/message/IntermediateMessageType.java
+++ /dev/null
@@ -1,46 +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.message;
-
-/**
- * The type of the intermediate stream message. The enum will be encoded using its ordinal value and
- * put in the first byte of the serialization of intermediate message.
- * For more details, see {@link org.apache.samza.serializers.IntermediateMessageSerde}
- */
-public enum IntermediateMessageType {
-  USER_MESSAGE,
-  WATERMARK_MESSAGE,
-  END_OF_STREAM_MESSAGE;
-
-  /**
-   * Returns the {@link IntermediateMessageType} of a particular intermediate stream message.
-   * @param message an intermediate stream message
-   * @return type of the message
-   */
-  public static IntermediateMessageType of(Object message) {
-    if (message instanceof WatermarkMessage) {
-      return WATERMARK_MESSAGE;
-    } else if (message instanceof EndOfStreamMessage) {
-      return END_OF_STREAM_MESSAGE;
-    } else {
-      return USER_MESSAGE;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/message/MessageType.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/message/MessageType.java b/samza-core/src/main/java/org/apache/samza/message/MessageType.java
deleted file mode 100644
index b1199b6..0000000
--- a/samza-core/src/main/java/org/apache/samza/message/MessageType.java
+++ /dev/null
@@ -1,46 +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.message;
-
-/**
- * The type of the intermediate stream message. The enum will be encoded using its ordinal value and
- * put in the first byte of the serialization of intermediate message.
- * For more details, see {@link org.apache.samza.serializers.IntermediateMessageSerde}
- */
-public enum MessageType {
-  USER_MESSAGE,
-  WATERMARK,
-  END_OF_STREAM;
-
-  /**
-   * Returns the {@link MessageType} of a particular intermediate stream message.
-   * @param message an intermediate stream message
-   * @return type of the message
-   */
-  public static MessageType of(Object message) {
-    if (message instanceof WatermarkMessage) {
-      return WATERMARK;
-    } else if (message instanceof EndOfStreamMessage) {
-      return END_OF_STREAM;
-    } else {
-      return USER_MESSAGE;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/message/WatermarkMessage.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/message/WatermarkMessage.java b/samza-core/src/main/java/org/apache/samza/message/WatermarkMessage.java
deleted file mode 100644
index aa25742..0000000
--- a/samza-core/src/main/java/org/apache/samza/message/WatermarkMessage.java
+++ /dev/null
@@ -1,43 +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.message;
-
-import org.codehaus.jackson.annotate.JsonCreator;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/**
- *  The WatermarkMessage is a control message that is sent out to next stage
- *  with a watermark timestamp and the task that produces the watermark.
- */
-public class WatermarkMessage extends ControlMessage {
-  private final long timestamp;
-
-  @JsonCreator
-  public WatermarkMessage(@JsonProperty("timestamp") long timestamp,
-                          @JsonProperty("task-name") String taskName,
-                          @JsonProperty("task-count") int taskCount) {
-    super(taskName, taskCount);
-    this.timestamp = timestamp;
-  }
-
-  public long getTimestamp() {
-    return timestamp;
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
index 2aec49f..2c2eb56 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
@@ -24,7 +24,6 @@ import org.apache.samza.operators.spec.InputOperatorSpec;
 import org.apache.samza.operators.spec.OutputStreamImpl;
 import org.apache.samza.operators.stream.IntermediateMessageStreamImpl;
 import org.apache.samza.operators.spec.OperatorSpec;
-import org.apache.samza.control.IOGraph;
 import org.apache.samza.runtime.ApplicationRunner;
 import org.apache.samza.system.StreamSpec;
 
@@ -206,8 +205,4 @@ public class StreamGraphImpl implements StreamGraph {
 
     return windowOrJoinSpecs.size() != 0;
   }
-
-  public IOGraph toIOGraph() {
-    return IOGraph.buildIOGraph(this);
-  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/operators/impl/ControlMessageSender.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/ControlMessageSender.java b/samza-core/src/main/java/org/apache/samza/operators/impl/ControlMessageSender.java
new file mode 100644
index 0000000..3bdc361
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/ControlMessageSender.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.impl;
+
+import org.apache.samza.system.ControlMessage;
+import org.apache.samza.system.MessageType;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.StreamMetadataCache;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamMetadata;
+import org.apache.samza.task.MessageCollector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a helper class to broadcast control messages to each partition of an intermediate stream
+ */
+class ControlMessageSender {
+  private static final Logger LOG = LoggerFactory.getLogger(ControlMessageSender.class);
+
+  private final StreamMetadataCache metadataCache;
+
+  ControlMessageSender(StreamMetadataCache metadataCache) {
+    this.metadataCache = metadataCache;
+  }
+
+  void send(ControlMessage message, SystemStream systemStream, MessageCollector collector) {
+    SystemStreamMetadata metadata = metadataCache.getSystemStreamMetadata(systemStream, true);
+    int partitionCount = metadata.getSystemStreamPartitionMetadata().size();
+    LOG.info(String.format("Broadcast %s message from task %s to %s with %s partition",
+        MessageType.of(message).name(), message.getTaskName(), systemStream, partitionCount));
+
+    for (int i = 0; i < partitionCount; i++) {
+      OutgoingMessageEnvelope envelopeOut = new OutgoingMessageEnvelope(systemStream, i, null, message);
+      collector.send(envelopeOut);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/operators/impl/EndOfStreamStates.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/EndOfStreamStates.java b/samza-core/src/main/java/org/apache/samza/operators/impl/EndOfStreamStates.java
new file mode 100644
index 0000000..a69b234
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/EndOfStreamStates.java
@@ -0,0 +1,98 @@
+/*
+ * 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 java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import org.apache.samza.system.EndOfStreamMessage;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+
+
+/**
+ * This class manages the end-of-stream state of the streams in a task. Internally it keeps track of end-of-stream
+ * messages received from upstream tasks for each system stream partition (ssp). If messages have been received from
+ * all tasks, it will mark the ssp as end-of-stream. For a stream to be end-of-stream, all its partitions assigned to
+ * the task need to be end-of-stream.
+ *
+ * This class is thread-safe.
+ */
+class EndOfStreamStates {
+
+  private static final class EndOfStreamState {
+    // set of upstream tasks
+    private final Set<String> tasks = new HashSet<>();
+    private final int expectedTotal;
+    private volatile boolean isEndOfStream = false;
+
+    EndOfStreamState(int expectedTotal) {
+      this.expectedTotal = expectedTotal;
+    }
+
+    synchronized void update(String taskName) {
+      if (taskName != null) {
+        tasks.add(taskName);
+      }
+      isEndOfStream = tasks.size() == expectedTotal;
+    }
+
+    boolean isEndOfStream() {
+      return isEndOfStream;
+    }
+  }
+
+  private final Map<SystemStreamPartition, EndOfStreamState> eosStates;
+
+  /**
+   * Constructing the end-of-stream states for a task
+   * @param ssps all the ssps assigned to this task
+   * @param producerTaskCounts mapping from a stream to the number of upstream tasks that produce to it
+   */
+  EndOfStreamStates(Set<SystemStreamPartition> ssps, Map<SystemStream, Integer> producerTaskCounts) {
+    Map<SystemStreamPartition, EndOfStreamState> states = new HashMap<>();
+    ssps.forEach(ssp -> {
+        states.put(ssp, new EndOfStreamState(producerTaskCounts.getOrDefault(ssp.getSystemStream(), 0)));
+      });
+    this.eosStates = Collections.unmodifiableMap(states);
+  }
+
+  /**
+   * Update the state upon receiving an end-of-stream message.
+   * @param eos message of {@link EndOfStreamMessage}
+   * @param ssp system stream partition
+   */
+  void update(EndOfStreamMessage eos, SystemStreamPartition ssp) {
+    EndOfStreamState state = eosStates.get(ssp);
+    state.update(eos.getTaskName());
+  }
+
+  boolean isEndOfStream(SystemStream systemStream) {
+    return eosStates.entrySet().stream()
+        .filter(entry -> entry.getKey().getSystemStream().equals(systemStream))
+        .allMatch(entry -> entry.getValue().isEndOfStream());
+  }
+
+  boolean allEndOfStream() {
+    return eosStates.values().stream().allMatch(EndOfStreamState::isEndOfStream);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/2819cbc7/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
index 8f51c5f..eefd4eb 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
@@ -24,22 +24,29 @@ import java.util.HashSet;
 import java.util.Set;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.MetricsConfig;
-import org.apache.samza.control.Watermark;
-import org.apache.samza.control.WatermarkManager;
+import org.apache.samza.container.TaskContextImpl;
+import org.apache.samza.container.TaskName;
+import org.apache.samza.operators.functions.WatermarkFunction;
+import org.apache.samza.system.EndOfStreamMessage;
 import org.apache.samza.metrics.Counter;
 import org.apache.samza.metrics.MetricsRegistry;
 import org.apache.samza.metrics.Timer;
 import org.apache.samza.operators.spec.OperatorSpec;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.system.WatermarkMessage;
 import org.apache.samza.task.MessageCollector;
 import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
 import org.apache.samza.util.HighResolutionClock;
-
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Abstract base class for all stream operator implementations.
  */
 public abstract class OperatorImpl<M, RM> {
+  private static final Logger LOG = LoggerFactory.getLogger(OperatorImpl.class);
   private static final String METRICS_GROUP = OperatorImpl.class.getName();
 
   private boolean initialized;
@@ -48,11 +55,18 @@ public abstract class OperatorImpl<M, RM> {
   private Counter numMessage;
   private Timer handleMessageNs;
   private Timer handleTimerNs;
-  private long inputWatermarkTime = WatermarkManager.TIME_NOT_EXIST;
-  private long outputWatermarkTime = WatermarkManager.TIME_NOT_EXIST;
+  private long inputWatermark = WatermarkStates.WATERMARK_NOT_EXIST;
+  private long outputWatermark = WatermarkStates.WATERMARK_NOT_EXIST;
+  private TaskName taskName;
 
   Set<OperatorImpl<RM, ?>> registeredOperators;
   Set<OperatorImpl<?, M>> prevOperators;
+  Set<SystemStream> inputStreams;
+
+  // end-of-stream states
+  private EndOfStreamStates eosStates;
+  // watermark states
+  private WatermarkStates watermarkStates;
 
   /**
    * Initialize this {@link OperatorImpl} and its user-defined functions.
@@ -74,10 +88,16 @@ public abstract class OperatorImpl<M, RM> {
     this.highResClock = createHighResClock(config);
     registeredOperators = new HashSet<>();
     prevOperators = new HashSet<>();
+    inputStreams = new HashSet<>();
     MetricsRegistry metricsRegistry = context.getMetricsRegistry();
     this.numMessage = metricsRegistry.newCounter(METRICS_GROUP, opName + "-messages");
     this.handleMessageNs = metricsRegistry.newTimer(METRICS_GROUP, opName + "-handle-message-ns");
     this.handleTimerNs = metricsRegistry.newTimer(METRICS_GROUP, opName + "-handle-timer-ns");
+    this.taskName = context.getTaskName();
+
+    TaskContextImpl taskContext = (TaskContextImpl) context;
+    this.eosStates = (EndOfStreamStates) taskContext.fetchObject(EndOfStreamStates.class.getName());
+    this.watermarkStates = (WatermarkStates) taskContext.fetchObject(WatermarkStates.class.getName());
 
     handleInit(config, context);
 
@@ -111,6 +131,10 @@ public abstract class OperatorImpl<M, RM> {
     this.prevOperators.add(prevOperator);
   }
 
+  void registerInputStream(SystemStream input) {
+    this.inputStreams.add(input);
+  }
+
   /**
    * Handle the incoming {@code message} for this {@link OperatorImpl} and propagate results to registered operators.
    * <p>
@@ -128,6 +152,13 @@ public abstract class OperatorImpl<M, RM> {
     this.handleMessageNs.update(endNs - startNs);
 
     results.forEach(rm -> this.registeredOperators.forEach(op -> op.onMessage(rm, collector, coordinator)));
+
+    WatermarkFunction watermarkFn = getOperatorSpec().getWatermarkFn();
+    if (watermarkFn != null) {
+      // check whether there is new watermark emitted from the user function
+      Long outputWm = watermarkFn.getOutputWatermark();
+      propagateWatermark(outputWm, collector, coordinator);
+    }
   }
 
   /**
@@ -174,68 +205,149 @@ public abstract class OperatorImpl<M, RM> {
   }
 
   /**
-   * Populate the watermarks based on the following equations:
-   *
-   * <ul>
-   *   <li>InputWatermark(op) = min { OutputWatermark(op') | op1 is upstream of op}</li>
-   *   <li>OutputWatermark(op) = min { InputWatermark(op), OldestWorkTime(op) }</li>
-   * </ul>
-   *
-   * @param watermark incoming watermark
+   * Aggregate {@link EndOfStreamMessage} from each ssp of the stream.
+   * Invoke onEndOfStream() if the stream reaches the end.
+   * @param eos {@link EndOfStreamMessage} object
+   * @param ssp system stream partition
    * @param collector message collector
    * @param coordinator task coordinator
    */
-  public final void onWatermark(Watermark watermark,
-      MessageCollector collector,
+  public final void aggregateEndOfStream(EndOfStreamMessage eos, SystemStreamPartition ssp, MessageCollector collector,
       TaskCoordinator coordinator) {
+    LOG.info("Received end-of-stream message from task {} in {}", eos.getTaskName(), ssp);
+    eosStates.update(eos, ssp);
+
+    SystemStream stream = ssp.getSystemStream();
+    if (eosStates.isEndOfStream(stream)) {
+      LOG.info("Input {} reaches the end for task {}", stream.toString(), taskName.getTaskName());
+      onEndOfStream(collector, coordinator);
+
+      if (eosStates.allEndOfStream()) {
+        // all inputs have been end-of-stream, shut down the task
+        LOG.info("All input streams have reached the end for task {}", taskName.getTaskName());
+        coordinator.shutdown(TaskCoordinator.RequestScope.CURRENT_TASK);
+      }
+    }
+  }
+
+  /**
+   * Invoke handleEndOfStream() if all the input streams to the current operator reach the end.
+   * Propagate the end-of-stream to downstream operators.
+   * @param collector message collector
+   * @param coordinator task coordinator
+   */
+  private final void onEndOfStream(MessageCollector collector, TaskCoordinator coordinator) {
+    if (inputStreams.stream().allMatch(input -> eosStates.isEndOfStream(input))) {
+      handleEndOfStream(collector, coordinator);
+      this.registeredOperators.forEach(op -> op.onEndOfStream(collector, coordinator));
+    }
+  }
+
+  /**
+   * All input streams to this operator reach to the end.
+   * Inherited class should handle end-of-stream by overriding this function.
+   * By default noop implementation is for in-memory operator to handle the EOS. Output operator need to
+   * override this to actually propagate EOS over the wire.
+   * @param collector message collector
+   * @param coordinator task coordinator
+   */
+  protected void handleEndOfStream(MessageCollector collector, TaskCoordinator coordinator) {
+    //Do nothing by default
+  }
+
+  /**
+   * Aggregate the {@link WatermarkMessage} from each ssp into a watermark. Then call onWatermark() if
+   * a new watermark exits.
+   * @param watermarkMessage a {@link WatermarkMessage} object
+   * @param ssp {@link SystemStreamPartition} that the message is coming from.
+   * @param collector message collector
+   * @param coordinator task coordinator
+   */
+  public final void aggregateWatermark(WatermarkMessage watermarkMessage, SystemStreamPartition ssp,
+      MessageCollector collector, TaskCoordinator coordinator) {
+    LOG.debug("Received watermark {} from {}", watermarkMessage.getTimestamp(), ssp);
+    watermarkStates.update(watermarkMessage, ssp);
+    long watermark = watermarkStates.getWatermark(ssp.getSystemStream());
+    if (watermark != WatermarkStates.WATERMARK_NOT_EXIST) {
+      LOG.debug("Got watermark {} from stream {}", watermark, ssp.getSystemStream());
+      onWatermark(watermark, collector, coordinator);
+    }
+  }
+
+  /**
+   * A watermark comes from an upstream operator. This function decides whether we should update the
+   * input watermark based on the watermark time of all the previous operators, and then call handleWatermark()
+   * to let the inherited operator to act on it.
+   * @param watermark incoming watermark from an upstream operator
+   * @param collector message collector
+   * @param coordinator task coordinator
+   */
+  private final void onWatermark(long watermark, MessageCollector collector, TaskCoordinator coordinator) {
     final long inputWatermarkMin;
     if (prevOperators.isEmpty()) {
       // for input operator, use the watermark time coming from the source input
-      inputWatermarkMin = watermark.getTimestamp();
+      inputWatermarkMin = watermark;
     } else {
-      // InputWatermark(op) = min { OutputWatermark(op') | op1 is upstream of op}
-      inputWatermarkMin = prevOperators.stream().map(op -> op.getOutputWatermarkTime()).min(Long::compare).get();
+      // InputWatermark(op) = min { OutputWatermark(op') | op' is upstream of op}
+      inputWatermarkMin = prevOperators.stream().map(op -> op.getOutputWatermark()).min(Long::compare).get();
     }
 
-    if (inputWatermarkTime < inputWatermarkMin) {
+    if (inputWatermark < inputWatermarkMin) {
       // advance the watermark time of this operator
-      inputWatermarkTime = inputWatermarkMin;
-      Watermark inputWatermark = watermark.copyWithTimestamp(inputWatermarkTime);
-      long oldestWorkTime = handleWatermark(inputWatermark, collector, coordinator);
-
-      // OutputWatermark(op) = min { InputWatermark(op), OldestWorkTime(op) }
-      long outputWatermarkMin = Math.min(inputWatermarkTime, oldestWorkTime);
-      if (outputWatermarkTime < outputWatermarkMin) {
-        // populate the watermark to downstream
-        outputWatermarkTime = outputWatermarkMin;
-        Watermark outputWatermark = watermark.copyWithTimestamp(outputWatermarkTime);
+      inputWatermark = inputWatermarkMin;
+      LOG.trace("Advance input watermark to {} in operator {}", inputWatermark, getOperatorName());
+
+      final Long outputWm;
+      WatermarkFunction watermarkFn = getOperatorSpec().getWatermarkFn();
+      if (watermarkFn != null) {
+        // user-overrided watermark handling here
+        watermarkFn.processWatermark(inputWatermark);
+        outputWm = watermarkFn.getOutputWatermark();
+      } else {
+        // use samza-provided watermark handling
+        // default is to propagate the input watermark
+        outputWm = handleWatermark(inputWatermark, collector, coordinator);
+      }
+
+      propagateWatermark(outputWm, collector, coordinator);
+    }
+  }
+
+  private void propagateWatermark(Long outputWm, MessageCollector collector, TaskCoordinator coordinator) {
+    if (outputWm != null) {
+      if (outputWatermark < outputWm) {
+        // advance the watermark
+        outputWatermark = outputWm;
+        LOG.debug("Advance output watermark to {} in operator {}", outputWatermark, getOperatorName());
         this.registeredOperators.forEach(op -> op.onWatermark(outputWatermark, collector, coordinator));
+      } else if (outputWatermark > outputWm) {
+        LOG.warn("Ignore watermark {} that is smaller than the previous watermark {}.", outputWm, outputWatermark);
       }
     }
   }
 
   /**
-   * Returns the oldest time of the envelops that haven't been processed by this operator
-   * Default implementation of handling watermark, which returns the input watermark time
-   * @param inputWatermark input watermark
+   * Handling of the input watermark and returns the output watermark.
+   * In-memory operator can override this to fire event-time triggers. Output operators need to override it
+   * so it can propagate watermarks over the wire. By default it simply returns the input watermark.
+   * @param inputWatermark  input watermark
    * @param collector message collector
    * @param coordinator task coordinator
-   * @return time of oldest processing envelope
+   * @return output watermark, or null if the output watermark should not be updated.
    */
-  protected long handleWatermark(Watermark inputWatermark,
-      MessageCollector collector,
-      TaskCoordinator coordinator) {
-    return inputWatermark.getTimestamp();
+  protected Long handleWatermark(long inputWatermark, MessageCollector collector, TaskCoordinator coordinator) {
+    // Default is no handling. Simply pass on the input watermark as output.
+    return inputWatermark;
   }
 
-  /* package private */
-  long getInputWatermarkTime() {
-    return this.inputWatermarkTime;
+  /* package private for testing */
+  final long getInputWatermark() {
+    return this.inputWatermark;
   }
 
-  /* package private */
-  long getOutputWatermarkTime() {
-    return this.outputWatermarkTime;
+  /* package private for testing */
+  final long getOutputWatermark() {
+    return this.outputWatermark;
   }
 
   public void close() {