You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by ni...@apache.org on 2018/05/25 16:36:44 UTC

[01/10] samza git commit: SAMZA-1659: Serializable OperatorSpec

Repository: samza
Updated Branches:
  refs/heads/master bc4a0c2de -> 53d7f2625


http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTable.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTable.java b/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTable.java
index 23fa9e6..d7f0570 100644
--- a/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTable.java
+++ b/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTable.java
@@ -21,11 +21,11 @@ package org.apache.samza.test.table;
 
 import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 
+import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JobConfig;
@@ -96,17 +96,59 @@ public class TestLocalTable extends AbstractIntegrationTestHarness {
     runner.run(app);
     runner.waitForFinish();
 
-    assertEquals(count * partitionCount, mapFn.received.size());
-    assertEquals(count, new HashSet(mapFn.received).size());
-    mapFn.received.forEach(p -> Assert.assertTrue(mapFn.table.get(p.getMemberId()) != null));
+    for (int i = 0; i < partitionCount; i++) {
+      MyMapFunction mapFnCopy = MyMapFunction.getMapFunctionByTask(String.format("Partition %d", i));
+      assertEquals(count, mapFnCopy.received.size());
+      mapFnCopy.received.forEach(p -> Assert.assertTrue(mapFnCopy.table.get(p.getMemberId()) != null));
+    }
+  }
+
+  static class TestStreamTableJoin {
+    static List<PageView> received = new LinkedList<>();
+    static List<EnrichedPageView> joined = new LinkedList<>();
+    final int count;
+    final int partitionCount;
+    final Map<String, String> configs;
+
+    TestStreamTableJoin(int count, int partitionCount, Map<String, String> configs) {
+      this.count = count;
+      this.partitionCount = partitionCount;
+      this.configs = configs;
+    }
+
+    void runTest() {
+      final LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs));
+      final StreamApplication app = (streamGraph, cfg) -> {
+
+        Table<KV<Integer, Profile>> table = streamGraph.getTable(
+            new InMemoryTableDescriptor("t1").withSerde(KVSerde.of(new IntegerSerde(), new ProfileJsonSerde())));
+
+        streamGraph.getInputStream("Profile", new NoOpSerde<Profile>())
+            .map(m -> new KV(m.getMemberId(), m))
+            .sendTo(table);
+
+        streamGraph.getInputStream("PageView", new NoOpSerde<PageView>())
+            .map(pv -> {
+                received.add(pv);
+                return pv;
+              })
+            .partitionBy(PageView::getMemberId, v -> v, "p1")
+            .join(table, new PageViewToProfileJoinFunction())
+            .sink((m, collector, coordinator) -> joined.add(m));
+      };
+
+      runner.run(app);
+      runner.waitForFinish();
+
+      assertEquals(count * partitionCount, received.size());
+      assertEquals(count * partitionCount, joined.size());
+      assertTrue(joined.get(0) instanceof EnrichedPageView);
+    }
   }
 
   @Test
   public void testStreamTableJoin() throws Exception {
 
-    List<PageView> received = new LinkedList<>();
-    List<EnrichedPageView> joined = new LinkedList<>();
-
     int count = 10;
     PageView[] pageViews = TestTableData.generatePageViews(count);
     Profile[] profiles = TestTableData.generateProfiles(count);
@@ -123,48 +165,89 @@ public class TestLocalTable extends AbstractIntegrationTestHarness {
     configs.put("streams.Profile.source", Base64Serializer.serialize(profiles));
     configs.put("streams.Profile.partitionCount", String.valueOf(partitionCount));
 
-    final LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs));
-    final StreamApplication app = (streamGraph, cfg) -> {
-
-      Table<KV<Integer, Profile>> table = streamGraph.getTable(new InMemoryTableDescriptor("t1")
-          .withSerde(KVSerde.of(new IntegerSerde(), new ProfileJsonSerde())));
+    TestStreamTableJoin joinTest = new TestStreamTableJoin(count, partitionCount, configs);
+    joinTest.runTest();
+  }
 
-      streamGraph.getInputStream("Profile", new NoOpSerde<Profile>())
-          .map(m -> new KV(m.getMemberId(), m))
-          .sendTo(table);
+  static class TestDualStreamTableJoin {
+    static List<Profile> sentToProfileTable1 = new LinkedList<>();
+    static List<Profile> sentToProfileTable2 = new LinkedList<>();
+    static List<EnrichedPageView> joinedPageViews1 = new LinkedList<>();
+    static List<EnrichedPageView> joinedPageViews2 = new LinkedList<>();
+    final int count;
+    final int partitionCount;
+    final Map<String, String> configs;
+
+    TestDualStreamTableJoin(int count, int partitionCount, Map<String, String> configs) {
+      this.count = count;
+      this.partitionCount = partitionCount;
+      this.configs = configs;
+    }
 
-      streamGraph.getInputStream("PageView", new NoOpSerde<PageView>())
-          .map(pv -> {
-              received.add(pv);
-              return pv;
-            })
-          .partitionBy(PageView::getMemberId, v -> v, "p1")
-          .join(table, new PageViewToProfileJoinFunction())
-          .sink((m, collector, coordinator) -> joined.add(m));
-    };
+    void runTest() {
+      KVSerde<Integer, Profile> profileKVSerde = KVSerde.of(new IntegerSerde(), new ProfileJsonSerde());
+      KVSerde<Integer, PageView> pageViewKVSerde = KVSerde.of(new IntegerSerde(), new PageViewJsonSerde());
+
+      PageViewToProfileJoinFunction joinFn1 = new PageViewToProfileJoinFunction();
+      PageViewToProfileJoinFunction joinFn2 = new PageViewToProfileJoinFunction();
+
+      final LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs));
+      final StreamApplication app = (streamGraph, cfg) -> {
+
+        Table<KV<Integer, Profile>> profileTable = streamGraph.getTable(new InMemoryTableDescriptor("t1")
+            .withSerde(profileKVSerde));
+
+        MessageStream<Profile> profileStream1 = streamGraph.getInputStream("Profile1", new NoOpSerde<Profile>());
+        MessageStream<Profile> profileStream2 = streamGraph.getInputStream("Profile2", new NoOpSerde<Profile>());
+
+        profileStream1
+            .map(m -> {
+                sentToProfileTable1.add(m);
+                return new KV(m.getMemberId(), m);
+              })
+            .sendTo(profileTable);
+        profileStream2
+            .map(m -> {
+                sentToProfileTable2.add(m);
+                return new KV(m.getMemberId(), m);
+              })
+            .sendTo(profileTable);
+
+        MessageStream<PageView> pageViewStream1 = streamGraph.getInputStream("PageView1", new NoOpSerde<PageView>());
+        MessageStream<PageView> pageViewStream2 = streamGraph.getInputStream("PageView2", new NoOpSerde<PageView>());
+
+        pageViewStream1
+            .partitionBy(PageView::getMemberId, v -> v, pageViewKVSerde, "p1")
+            .join(profileTable, joinFn1)
+            .sink((m, collector, coordinator) -> joinedPageViews1.add(m));
+
+        pageViewStream2
+            .partitionBy(PageView::getMemberId, v -> v, pageViewKVSerde, "p2")
+            .join(profileTable, joinFn2)
+            .sink((m, collector, coordinator) -> joinedPageViews2.add(m));
+      };
+
+      runner.run(app);
+      runner.waitForFinish();
+
+      assertEquals(count * partitionCount, sentToProfileTable1.size());
+      assertEquals(count * partitionCount, sentToProfileTable2.size());
+
+      for (int i = 0; i < PageViewToProfileJoinFunction.seqNo; i++) {
+        assertEquals(count * partitionCount, PageViewToProfileJoinFunction.counterPerJoinFn.get(i).intValue());
+      }
+      assertEquals(count * partitionCount, joinedPageViews1.size());
+      assertEquals(count * partitionCount, joinedPageViews2.size());
+      assertTrue(joinedPageViews1.get(0) instanceof EnrichedPageView);
+      assertTrue(joinedPageViews2.get(0) instanceof EnrichedPageView);
 
-    runner.run(app);
-    runner.waitForFinish();
+    }
 
-    assertEquals(count * partitionCount, received.size());
-    assertEquals(count * partitionCount, joined.size());
-    assertTrue(joined.get(0) instanceof EnrichedPageView);
   }
 
   @Test
   public void testDualStreamTableJoin() throws Exception {
 
-    List<Profile> sentToProfileTable1 = new LinkedList<>();
-    List<Profile> sentToProfileTable2 = new LinkedList<>();
-    List<EnrichedPageView> joinedPageViews1 = new LinkedList<>();
-    List<EnrichedPageView> joinedPageViews2 = new LinkedList<>();
-
-    KVSerde<Integer, Profile> profileKVSerde = KVSerde.of(new IntegerSerde(), new ProfileJsonSerde());
-    KVSerde<Integer, PageView> pageViewKVSerde = KVSerde.of(new IntegerSerde(), new PageViewJsonSerde());
-
-    PageViewToProfileJoinFunction joinFn1 = new PageViewToProfileJoinFunction();
-    PageViewToProfileJoinFunction joinFn2 = new PageViewToProfileJoinFunction();
-
     int count = 10;
     PageView[] pageViews = TestTableData.generatePageViews(count);
     Profile[] profiles = TestTableData.generateProfiles(count);
@@ -190,53 +273,8 @@ public class TestLocalTable extends AbstractIntegrationTestHarness {
     configs.put("streams.PageView2.source", Base64Serializer.serialize(pageViews));
     configs.put("streams.PageView2.partitionCount", String.valueOf(partitionCount));
 
-    final LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs));
-    final StreamApplication app = (streamGraph, cfg) -> {
-
-      Table<KV<Integer, Profile>> profileTable = streamGraph.getTable(new InMemoryTableDescriptor("t1")
-          .withSerde(KVSerde.of(new IntegerSerde(), new ProfileJsonSerde())));
-
-      MessageStream<Profile> profileStream1 = streamGraph.getInputStream("Profile1", new NoOpSerde<Profile>());
-      MessageStream<Profile> profileStream2 = streamGraph.getInputStream("Profile2", new NoOpSerde<Profile>());
-
-      profileStream1
-          .map(m -> {
-              sentToProfileTable1.add(m);
-              return new KV(m.getMemberId(), m);
-            })
-          .sendTo(profileTable);
-      profileStream2
-          .map(m -> {
-              sentToProfileTable2.add(m);
-              return new KV(m.getMemberId(), m);
-            })
-          .sendTo(profileTable);
-
-      MessageStream<PageView> pageViewStream1 = streamGraph.getInputStream("PageView1", new NoOpSerde<PageView>());
-      MessageStream<PageView> pageViewStream2 = streamGraph.getInputStream("PageView2", new NoOpSerde<PageView>());
-
-      pageViewStream1
-          .partitionBy(PageView::getMemberId, v -> v, pageViewKVSerde, "p1")
-          .join(profileTable, joinFn1)
-          .sink((m, collector, coordinator) -> joinedPageViews1.add(m));
-
-      pageViewStream2
-          .partitionBy(PageView::getMemberId, v -> v, pageViewKVSerde, "p2")
-          .join(profileTable, joinFn2)
-          .sink((m, collector, coordinator) -> joinedPageViews2.add(m));
-    };
-
-    runner.run(app);
-    runner.waitForFinish();
-
-    assertEquals(count * partitionCount, sentToProfileTable1.size());
-    assertEquals(count * partitionCount, sentToProfileTable2.size());
-    assertEquals(count * partitionCount, joinFn1.count);
-    assertEquals(count * partitionCount, joinFn2.count);
-    assertEquals(count * partitionCount, joinedPageViews1.size());
-    assertEquals(count * partitionCount, joinedPageViews2.size());
-    assertTrue(joinedPageViews1.get(0) instanceof EnrichedPageView);
-    assertTrue(joinedPageViews2.get(0) instanceof EnrichedPageView);
+    TestDualStreamTableJoin dualJoinTest = new TestDualStreamTableJoin(count, partitionCount, configs);
+    dualJoinTest.runTest();
   }
 
   static Map<String, String> getBaseJobConfig(String bootstrapUrl, String zkConnect) {
@@ -264,14 +302,19 @@ public class TestLocalTable extends AbstractIntegrationTestHarness {
     return configs;
   }
 
-  private class MyMapFunction implements MapFunction<Profile, KV<Integer, Profile>> {
+  private static class MyMapFunction implements MapFunction<Profile, KV<Integer, Profile>> {
 
-    private List<Profile> received = new ArrayList<>();
-    private ReadableTable table;
+    private static Map<String, MyMapFunction> taskToMapFunctionMap = new HashMap<>();
+
+    private transient List<Profile> received;
+    private transient ReadableTable table;
 
     @Override
     public void init(Config config, TaskContext context) {
       table = (ReadableTable) context.getTable("t1");
+      this.received = new ArrayList<>();
+
+      taskToMapFunctionMap.put(context.getTaskName().getTaskName(), this);
     }
 
     @Override
@@ -279,14 +322,30 @@ public class TestLocalTable extends AbstractIntegrationTestHarness {
       received.add(profile);
       return new KV(profile.getMemberId(), profile);
     }
+
+    public static MyMapFunction getMapFunctionByTask(String taskName) {
+      return taskToMapFunctionMap.get(taskName);
+    }
   }
 
   static class PageViewToProfileJoinFunction implements StreamTableJoinFunction
       <Integer, KV<Integer, PageView>, KV<Integer, Profile>, EnrichedPageView> {
-    private int count;
+    private static Map<Integer, AtomicInteger> counterPerJoinFn = new HashMap<>();
+    private static int seqNo = 0;
+    private final int currentSeqNo;
+
+    public PageViewToProfileJoinFunction() {
+      this.currentSeqNo = seqNo++;
+    }
+
+    @Override
+    public void init(Config config, TaskContext context) {
+      counterPerJoinFn.put(this.currentSeqNo, new AtomicInteger(0));
+    }
+
     @Override
     public EnrichedPageView apply(KV<Integer, PageView> m, KV<Integer, Profile> r) {
-      ++count;
+      counterPerJoinFn.get(this.currentSeqNo).incrementAndGet();
       return r == null ? null :
           new EnrichedPageView(m.getValue().getPageKey(), m.getKey(), r.getValue().getCompany());
     }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java b/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java
index a260c3f..208c670 100644
--- a/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java
+++ b/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java
@@ -60,15 +60,33 @@ import static org.mockito.Mockito.mock;
 
 
 public class TestRemoteTable extends AbstractIntegrationTestHarness {
-  private TableReadFunction<Integer, TestTableData.Profile> getInMemoryReader(TestTableData.Profile[] profiles) {
-    final Map<Integer, TestTableData.Profile> profileMap = Arrays.stream(profiles)
-        .collect(Collectors.toMap(p -> p.getMemberId(), Function.identity()));
-    TableReadFunction<Integer, TestTableData.Profile> reader =
-        (TableReadFunction<Integer, TestTableData.Profile>) key -> profileMap.getOrDefault(key, null);
-    return reader;
-  }
 
   static List<TestTableData.EnrichedPageView> writtenRecords = new LinkedList<>();
+  static List<TestTableData.PageView> received = new LinkedList<>();
+
+  static class InMemoryReadFunction implements TableReadFunction<Integer, TestTableData.Profile> {
+    private final String serializedProfiles;
+    private transient Map<Integer, TestTableData.Profile> profileMap;
+
+    private InMemoryReadFunction(String profiles) {
+      this.serializedProfiles = profiles;
+    }
+
+    private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+      in.defaultReadObject();
+      TestTableData.Profile[] profiles = Base64Serializer.deserialize(this.serializedProfiles, TestTableData.Profile[].class);
+      this.profileMap = Arrays.stream(profiles).collect(Collectors.toMap(p -> p.getMemberId(), Function.identity()));
+    }
+
+    @Override
+    public TestTableData.Profile get(Integer key) {
+      return profileMap.getOrDefault(key, null);
+    }
+
+    static InMemoryReadFunction getInMemoryReadFunction(String serializedProfiles) {
+      return new InMemoryReadFunction(serializedProfiles);
+    }
+  }
 
   static class InMemoryWriteFunction implements TableWriteFunction<Integer, TestTableData.EnrichedPageView> {
     private transient List<TestTableData.EnrichedPageView> records;
@@ -99,12 +117,11 @@ public class TestRemoteTable extends AbstractIntegrationTestHarness {
 
   @Test
   public void testStreamTableJoinRemoteTable() throws Exception {
-    List<TestTableData.PageView> received = new LinkedList<>();
     final InMemoryWriteFunction writer = new InMemoryWriteFunction();
 
     int count = 10;
     TestTableData.PageView[] pageViews = TestTableData.generatePageViews(count);
-    TestTableData.Profile[] profiles = TestTableData.generateProfiles(count);
+    String profiles = Base64Serializer.serialize(TestTableData.generateProfiles(count));
 
     int partitionCount = 4;
     Map<String, String> configs = TestLocalTable.getBaseJobConfig(bootstrapUrl(), zkConnect());
@@ -119,7 +136,7 @@ public class TestRemoteTable extends AbstractIntegrationTestHarness {
     final StreamApplication app = (streamGraph, cfg) -> {
       RemoteTableDescriptor<Integer, TestTableData.Profile> inputTableDesc = new RemoteTableDescriptor<>("profile-table-1");
       inputTableDesc
-          .withReadFunction(getInMemoryReader(profiles))
+          .withReadFunction(InMemoryReadFunction.getInMemoryReadFunction(profiles))
           .withRateLimiter(readRateLimiter, null, null);
 
       RemoteTableDescriptor<Integer, TestTableData.EnrichedPageView> outputTableDesc = new RemoteTableDescriptor<>("enriched-page-view-table-1");

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/test/java/org/apache/samza/test/timer/TestTimerApp.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/timer/TestTimerApp.java b/samza-test/src/test/java/org/apache/samza/test/timer/TestTimerApp.java
index 27d1063..94c1eca 100644
--- a/samza-test/src/test/java/org/apache/samza/test/timer/TestTimerApp.java
+++ b/samza-test/src/test/java/org/apache/samza/test/timer/TestTimerApp.java
@@ -57,12 +57,13 @@ public class TestTimerApp implements StreamApplication {
 
   private static class FlatmapTimerFn implements FlatMapFunction<PageView, PageView>, TimerFunction<String, PageView> {
 
-    private List<PageView> pageViews = new ArrayList<>();
-    private TimerRegistry<String> timerRegistry;
+    private transient List<PageView> pageViews;
+    private transient TimerRegistry<String> timerRegistry;
 
     @Override
     public void registerTimer(TimerRegistry<String> timerRegistry) {
       this.timerRegistry = timerRegistry;
+      this.pageViews = new ArrayList<>();
     }
 
     @Override


[10/10] samza git commit: SAMZA-1659: Serializable OperatorSpec

Posted by ni...@apache.org.
SAMZA-1659: Serializable OperatorSpec

This change is to make the user supplied functions serializable. Hence, making the full user defined DAG serializable.

Author: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Author: Yi Pan (Data Infrastructure) <yi...@yipan-mn1.linkedin.biz>
Author: Xinyu Liu <xi...@xiliu-ld.linkedin.biz>

Reviewers: Jagadish <jv...@linkedin.com>, Prateek Maheshwari <pm...@linkedin.com>

Closes #475 from nickpan47/serializable-opspec-only-Jan-24-18 and squashes the following commits:

db0dea73 [Yi Pan (Data Infrastructure)] SAMZA-1659: fix intermittent TestZkLocalApplicationRunner failure due to StreamProcessor#stop()
34716d42 [Yi Pan (Data Infrastructure)] SAMZA-1659: fix a comment on OperatorSpec#isClone()
37d4e6ae [Yi Pan (Data Infrastructure)] SAMZA-1659: addressing latest round of review comments
68674a14 [Yi Pan (Data Infrastructure)] Merge branch 'master' into serializable-opspec-only-Jan-24-18
d3a7826c [Yi Pan (Data Infrastructure)] SAMZA-1659: addressing review comments
f83e8dd0 [Yi Pan (Data Infrastructure)] Merge branch 'master' into serializable-opspec-only-Jan-24-18
acca418b [Yi Pan (Data Infrastructure)] Merge branch 'master' into serializable-opspec-only-Jan-24-18
842a73d6 [Yi Pan (Data Infrastructure)] SAMZA-1659: making user-defined functions in high-level API serializable
ad85a2cb [Yi Pan (Data Infrastructure)] Merge branch 'master' into serializable-opspec-only-Jan-24-18
c1567116 [Yi Pan (Data Infrastructure)] SAMZA-1659: Before re-merge with master. Still need to fix unit tests (moving OperatorSpec clone tests to OperatorSpecGraph.clone)
f2563f8e [Yi Pan (Data Infrastructure)] SAMZA-1659: serialize the whole DAG instead of each individual OperatorSpec.
24d33496 [Yi Pan (Data Infrastructure)] SAMZA-1659: updated according to review comments. Need to merge again with master.
3f643f8b [Yi Pan (Data Infrastructure)] SAMZA-1659: serialiable OperatorSpec
ed7d8c0e [Yi Pan (Data Infrastructure)] Fixed some javadoc and test files
94de218b [Yi Pan (Data Infrastructure)] Remove public access from StreamGraphImpl#getIntermediateStream(String, Serde)
8f4e9dd4 [Yi Pan (Data Infrastructure)] Serialization of StreamGraph in a wrapper class SerializedStreamGraph
f3bb1958 [Yi Pan (Data Infrastructure)] Fix some comments
c15246f5 [Yi Pan (Data Infrastructure)] Merge branch 'master' into serializable-opspec-only-Jan-24-18
e981967d [Yi Pan (Data Infrastructure)] WIP: fixing unit test for SamzaSQL translators w/ serialization of operator functions
40583051 [Yi Pan (Data Infrastructure)] WIP: update the serialization of user functions after the merge
18ba924f [Yi Pan (Data Infrastructure)] Merge branch 'master' into serializable-opspec-only-Jan-24-18
93951c5f [Yi Pan (Data Infrastructure)] Merge branch 'master' into serializable-opspec-only-Jan-24-18
54a28801 [Yi Pan (Data Infrastructure)] WIP: broadcast, sendtotable, and streamtotablejoin serialization and unit tests
45eb1fb0 [Yi Pan (Data Infrastructure)] Merge branch 'master' into serializable-opspec-only-Jan-24-18
7c8d1591 [Yi Pan (Data Infrastructure)] WIP: working on unit tests for trigger, broadcast, join, table, and SQL UDF function serialization
b973b105 [Yi Pan (Data Infrastructure)] Merge branch 'master' into serializable-opspec-only-Jan-24-18
aca42308 [Yi Pan (Data Infrastructure)] WIP: Serialize OperatorSpec only w/o StreamApplication interface change. Passed all build and tests.
0ebebfc3 [Yi Pan (Data Infrastructure)] WIP: serialization only change
1670aff0 [Yi Pan (Data Infrastructure)] WIP: class-loading of user program logic and main() method based user program logic are both included in ThreadJobFactory/ProcessJobFactory/YarnJobFactory. ThreadJobFactory test suite to be fixed.
4102aa8c [Yi Pan (Data Infrastructure)] WIP: continued working on potential offspring integration
dc7da87e [Yi Pan (Data Infrastructure)] WIP: unit tests for serialization
475a46bc [Yi Pan (Data Infrastructure)] WIP: fixed TestZkLocalApplicationRunner. Debugging issues w/ TestRepartitionWindowApp (i.e. missing changelog creation step when directly running LocalApplicationRunner)
6a14b2af [Yi Pan (Data Infrastructure)] WIP: fixed unit test failure for Windows
d4640329 [Yi Pan (Data Infrastructure)] WIP: fixing unit tests after merge
bf1ce907 [Yi Pan (Data Infrastructure)] WIP: removing StreamDescriptor first
50201728 [Yi Pan (Data Infrastructure)] Merge branch 'experiment-new-api-v2' into new-api-v2-0.14
dde1ab14 [Yi Pan (Data Infrastructure)] WIP: first end-to-end test
d7df6ed0 [Yi Pan (Data Infrastructure)] WIP: added all unit test for OperatorSpec#copy methods.
6fc6d4c0 [Yi Pan (Data Infrastructure)] WIP: experiment code to implement an end-to-end working example for new APIs
525d8bc1 [Yi Pan (Data Infrastructure)] Merge branch '0.14.0' into new-api-v2
e6fb96e5 [Yi Pan (Data Infrastructure)] WIP: merged all application types into StreamApplications
f227380f [Yi Pan (Data Infrastructure)] WIP: update the app runner classes
256155ad [Yi Pan (Data Infrastructure)] WIP: new API user code examples
4a6a58dc [Yi Pan (Data Infrastructure)] WIP: updated w/ low-level task API and global var ingestion/metrics reporter
3c50629e [Yi Pan (Data Infrastructure)] WIP: adding support for low-level task APIs
51541e13 [Yi Pan (Data Infrastructure)] WIP: cleanup StreamDescriptor
0bc7ee7b [Yi Pan (Data Infrastructure)] WIP: update the user code example on new APIs
cd528c1c [Yi Pan (Data Infrastructure)] WIP: updated spec and user DAG API
b898e6c0 [Yi Pan (Data Infrastructure)] WIP: new-api-v2
91f364f1 [Yi Pan (Data Infrastructure)] WIP: proto-type of input/output stream/system specs
ae3dc6ff [Yi Pan (Data Infrastructure)] WIP: new api revision
8bb97520 [Yi Pan (Data Infrastructure)] WIP: proto-type of input/output stream/system specs
5573a069 [Yi Pan (Data Infrastructure)] WIP: new api revision
aeb45730 [Xinyu Liu] SAMZA-1321: Propagate end-of-stream and watermark messages


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

Branch: refs/heads/master
Commit: 53d7f2625145f560eb6ccc49d48dc176f244f9b3
Parents: bc4a0c2
Author: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Authored: Fri May 25 09:37:55 2018 -0700
Committer: Yi Pan (Data Infrastructure) <yi...@yipan-mn1.linkedin.biz>
Committed: Fri May 25 09:37:55 2018 -0700

----------------------------------------------------------------------
 build.gradle                                    |   5 +-
 .../samza/application/StreamApplication.java    |   7 +-
 .../java/org/apache/samza/config/MapConfig.java |   7 +-
 .../apache/samza/operators/MessageStream.java   |  19 +-
 .../operators/functions/ClosableFunction.java   |   3 +
 .../operators/functions/FilterFunction.java     |   3 +-
 .../operators/functions/FlatMapFunction.java    |   3 +-
 .../operators/functions/FoldLeftFunction.java   |  16 +-
 .../samza/operators/functions/JoinFunction.java |   3 +-
 .../samza/operators/functions/MapFunction.java  |   3 +-
 .../samza/operators/functions/SinkFunction.java |   3 +-
 .../functions/StreamTableJoinFunction.java      |   3 +-
 .../operators/functions/SupplierFunction.java   |  38 ++
 .../samza/operators/triggers/AnyTrigger.java    |  10 +-
 .../samza/operators/triggers/Trigger.java       |   3 +-
 .../apache/samza/operators/windows/Window.java  |   3 +-
 .../apache/samza/operators/windows/Windows.java |  53 +-
 .../windows/internal/WindowInternal.java        |  32 +-
 .../samza/serializers/SerializableSerde.java    |   2 +-
 .../org/apache/samza/system/StreamSpec.java     |   3 +-
 .../samza/system/SystemStreamPartition.java     |   4 +-
 .../java/org/apache/samza/table/TableSpec.java  |  12 +-
 .../samza/operators/windows/TestWindowPane.java |   2 +-
 .../samza/execution/ExecutionPlanner.java       |  28 +-
 .../org/apache/samza/execution/JobGraph.java    |  14 +-
 .../samza/execution/JobGraphJsonGenerator.java  |   4 +-
 .../org/apache/samza/execution/JobNode.java     |  26 +-
 .../samza/operators/MessageStreamImpl.java      |  79 ++-
 .../samza/operators/OperatorSpecGraph.java      | 132 ++++
 .../apache/samza/operators/StreamGraphImpl.java | 328 ----------
 .../apache/samza/operators/StreamGraphSpec.java | 299 +++++++++
 .../org/apache/samza/operators/TableImpl.java   |   3 +-
 .../operators/impl/BroadcastOperatorImpl.java   |   2 +-
 .../samza/operators/impl/OperatorImpl.java      |   2 +-
 .../samza/operators/impl/OperatorImplGraph.java |  73 ++-
 .../operators/impl/OutputOperatorImpl.java      |   5 +-
 .../operators/impl/PartitionByOperatorImpl.java |  16 +-
 .../operators/impl/StreamOperatorImpl.java      |   3 +-
 .../operators/impl/WindowOperatorImpl.java      |  23 +-
 .../operators/spec/FilterOperatorSpec.java      |  74 +++
 .../operators/spec/FlatMapOperatorSpec.java     |  47 ++
 .../samza/operators/spec/InputOperatorSpec.java |  13 +-
 .../samza/operators/spec/JoinOperatorSpec.java  |  17 +-
 .../samza/operators/spec/MapOperatorSpec.java   |  77 +++
 .../samza/operators/spec/MergeOperatorSpec.java |  51 ++
 .../samza/operators/spec/OperatorSpec.java      |  23 +-
 .../samza/operators/spec/OperatorSpecs.java     |  73 +--
 .../samza/operators/spec/OutputStreamImpl.java  |  17 +-
 .../operators/spec/PartitionByOperatorSpec.java |  23 +-
 .../operators/spec/SendToTableOperatorSpec.java |   9 +-
 .../operators/spec/StreamOperatorSpec.java      |  23 +-
 .../operators/spec/WindowOperatorSpec.java      |  11 +
 .../stream/IntermediateMessageStreamImpl.java   |   4 +-
 .../samza/operators/triggers/Cancellable.java   |   2 +-
 .../samza/operators/triggers/TriggerImpl.java   |   6 +-
 .../runtime/AbstractApplicationRunner.java      |  21 +-
 .../samza/runtime/LocalApplicationRunner.java   |  28 +-
 .../samza/runtime/LocalContainerRunner.java     |  11 +-
 .../apache/samza/task/StreamOperatorTask.java   |  48 +-
 .../org/apache/samza/task/TaskFactoryUtil.java  |  26 +-
 .../apache/samza/container/TaskInstance.scala   |   1 -
 .../samza/job/local/ThreadJobFactory.scala      |  11 +-
 .../apache/samza/example/BroadcastExample.java  |  71 ---
 .../samza/example/KeyValueStoreExample.java     | 131 ----
 .../org/apache/samza/example/MergeExample.java  |  60 --
 .../samza/example/OrderShipmentJoinExample.java | 115 ----
 .../samza/example/PageViewCounterExample.java   |  95 ---
 .../samza/example/RepartitionExample.java       |  90 ---
 .../org/apache/samza/example/WindowExample.java |  81 ---
 .../samza/execution/TestExecutionPlanner.java   | 117 ++--
 .../apache/samza/execution/TestJobGraph.java    |  68 +--
 .../execution/TestJobGraphJsonGenerator.java    |  32 +-
 .../org/apache/samza/execution/TestJobNode.java |  14 +-
 .../samza/operators/TestJoinOperator.java       | 152 ++---
 .../samza/operators/TestMessageStreamImpl.java  |  55 +-
 .../samza/operators/TestOperatorSpecGraph.java  | 185 ++++++
 .../samza/operators/TestStreamGraphImpl.java    | 601 -------------------
 .../samza/operators/TestStreamGraphSpec.java    | 601 +++++++++++++++++++
 .../data/TestOutputMessageEnvelope.java         |  14 +
 .../operators/impl/TestOperatorImplGraph.java   | 298 ++++++---
 .../operators/impl/TestStreamOperatorImpl.java  |   4 +-
 .../operators/impl/TestWindowOperator.java      | 263 ++++----
 .../operators/spec/OperatorSpecTestUtils.java   | 141 +++++
 .../samza/operators/spec/TestOperatorSpec.java  | 465 ++++++++++++++
 .../spec/TestPartitionByOperatorSpec.java       | 165 +++++
 .../operators/spec/TestWindowOperatorSpec.java  | 306 +++++++++-
 .../runtime/TestAbstractApplicationRunner.java  |  36 +-
 .../runtime/TestLocalApplicationRunner.java     |  21 +-
 .../apache/samza/task/IdentityStreamTask.java   |  55 ++
 .../apache/samza/task/TestTaskFactoryUtil.java  |  64 +-
 .../testUtils/InvalidStreamApplication.java     |  25 -
 .../samza/system/kafka/TestKafkaStreamSpec.java |   3 +-
 .../samza/sql/data/SamzaSqlCompositeKey.java    |   1 +
 .../sql/data/SamzaSqlExecutionContext.java      |  20 +-
 .../samza/sql/translator/FilterTranslator.java  |  47 +-
 .../translator/LogicalAggregateTranslator.java  |  24 +-
 .../samza/sql/translator/ProjectTranslator.java |  60 +-
 .../samza/sql/translator/QueryTranslator.java   |  46 +-
 .../SamzaSqlRelMessageJoinFunction.java         |  12 +-
 .../samza/sql/translator/ScanTranslator.java    |  28 +-
 .../samza/sql/translator/TranslatorContext.java |  79 ++-
 .../apache/samza/sql/TestQueryTranslator.java   | 510 ----------------
 .../sql/TestSamzaSqlApplicationConfig.java      |  95 ---
 .../sql/TestSamzaSqlApplicationRunner.java      |  56 --
 .../samza/sql/TestSamzaSqlFileParser.java       |  58 --
 .../samza/sql/TestSamzaSqlQueryParser.java      |  76 ---
 .../samza/sql/TestSamzaSqlRelMessage.java       |  46 --
 .../sql/TestSamzaSqlRelMessageJoinFunction.java | 119 ----
 .../samza/sql/data/TestSamzaSqlRelMessage.java  |  46 ++
 .../runner/TestSamzaSqlApplicationConfig.java   |  95 +++
 .../runner/TestSamzaSqlApplicationRunner.java   |  56 ++
 .../sql/testutil/TestSamzaSqlFileParser.java    |  58 ++
 .../sql/testutil/TestSamzaSqlQueryParser.java   |  75 +++
 .../sql/translator/TestFilterTranslator.java    | 136 +++++
 .../sql/translator/TestJoinTranslator.java      | 191 ++++++
 .../sql/translator/TestProjectTranslator.java   | 289 +++++++++
 .../sql/translator/TestQueryTranslator.java     | 596 ++++++++++++++++++
 .../TestSamzaSqlRelMessageJoinFunction.java     | 118 ++++
 .../sql/translator/TranslatorTestBase.java      |  72 +++
 .../example/AppWithGlobalConfigExample.java     |  86 +++
 .../apache/samza/example/BroadcastExample.java  |  70 +++
 .../samza/example/KeyValueStoreExample.java     | 138 +++++
 .../org/apache/samza/example/MergeExample.java  |  62 ++
 .../samza/example/OrderShipmentJoinExample.java | 121 ++++
 .../samza/example/PageViewCounterExample.java   | 100 +++
 .../samza/example/RepartitionExample.java       |  96 +++
 .../org/apache/samza/example/WindowExample.java |  86 +++
 .../samza/test/framework/StreamAssert.java      |  14 +
 .../EndOfStreamIntegrationTest.java             |   8 +-
 .../WatermarkIntegrationTest.java               |   7 +-
 .../test/operator/RepartitionJoinWindowApp.java |  54 +-
 .../test/operator/RepartitionWindowApp.java     |  72 +++
 .../samza/test/operator/SessionWindowApp.java   |  21 +-
 .../operator/TestRepartitionJoinWindowApp.java  |   2 +-
 .../test/operator/TestRepartitionWindowApp.java |  90 +++
 .../samza/test/operator/TumblingWindowApp.java  |  20 +-
 .../samza/test/operator/data/PageView.java      |   3 +-
 .../test/processor/SharedContextFactories.java  | 117 ++++
 .../test/processor/TestStreamApplication.java   | 148 +++++
 .../processor/TestZkLocalApplicationRunner.java | 211 +++----
 .../apache/samza/test/table/TestLocalTable.java | 243 +++++---
 .../samza/test/table/TestRemoteTable.java       |  37 +-
 .../apache/samza/test/timer/TestTimerApp.java   |   5 +-
 143 files changed, 7227 insertions(+), 3811 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 6872354..a94fcfa 100644
--- a/build.gradle
+++ b/build.gradle
@@ -325,6 +325,9 @@ project(':samza-sql') {
 
     testCompile "junit:junit:$junitVersion"
     testCompile "org.mockito:mockito-core:$mockitoVersion"
+    testCompile "org.powermock:powermock-api-mockito:$powerMockVersion"
+    testCompile "org.powermock:powermock-core:$powerMockVersion"
+    testCompile "org.powermock:powermock-module-junit4:$powerMockVersion"
 
     testRuntime "org.slf4j:slf4j-simple:$slf4jVersion"
   }
@@ -756,10 +759,10 @@ project(":samza-test_$scalaVersion") {
     compile project(":samza-kv-inmemory_$scalaVersion")
     compile project(":samza-kv-rocksdb_$scalaVersion")
     compile project(":samza-core_$scalaVersion")
+    compile project(":samza-kafka_$scalaVersion")
     compile project(":samza-sql")
     runtime project(":samza-log4j")
     runtime project(":samza-yarn_$scalaVersion")
-    runtime project(":samza-kafka_$scalaVersion")
     runtime project(":samza-hdfs_$scalaVersion")
     compile "org.scala-lang:scala-library:$scalaLibVersion"
     compile "net.sf.jopt-simple:jopt-simple:$joptSimpleVersion"

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-api/src/main/java/org/apache/samza/application/StreamApplication.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/application/StreamApplication.java b/samza-api/src/main/java/org/apache/samza/application/StreamApplication.java
index f615207..0b2142b 100644
--- a/samza-api/src/main/java/org/apache/samza/application/StreamApplication.java
+++ b/samza-api/src/main/java/org/apache/samza/application/StreamApplication.java
@@ -61,9 +61,10 @@ import org.apache.samza.task.TaskContext;
  *
  * <p>
  * Implementation Notes: Currently StreamApplications are wrapped in a {@link StreamTask} during execution.
- * A new StreamApplication instance will be created and initialized when planning the execution, as well as for each
- * {@link StreamTask} instance used for processing incoming messages. Execution is synchronous and thread-safe within
- * each {@link StreamTask}.
+ * A new StreamApplication instance will be created and initialized with a user-defined {@link StreamGraph}
+ * when planning the execution. The {@link StreamGraph} and the functions implemented for transforms are required to
+ * be serializable. The execution planner will generate a serialized DAG which will be deserialized in each {@link StreamTask}
+ * instance used for processing incoming messages. Execution is synchronous and thread-safe within each {@link StreamTask}.
  *
  * <p>
  * Functions implemented for transforms in StreamApplications ({@link org.apache.samza.operators.functions.MapFunction},

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-api/src/main/java/org/apache/samza/config/MapConfig.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/config/MapConfig.java b/samza-api/src/main/java/org/apache/samza/config/MapConfig.java
index 0b1ed98..5af2535 100644
--- a/samza-api/src/main/java/org/apache/samza/config/MapConfig.java
+++ b/samza-api/src/main/java/org/apache/samza/config/MapConfig.java
@@ -43,8 +43,11 @@ public class MapConfig extends Config {
 
   public MapConfig(List<Map<String, String>> maps) {
     this.map = new HashMap<>();
-    for (Map<String, String> m: maps)
-      this.map.putAll(m);
+    for (Map<String, String> m: maps) {
+      if (m != null) {
+        this.map.putAll(m);
+      }
+    }
   }
 
   public MapConfig(Map<String, String>... maps) {

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java
index 98f0784..7797f9a 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java
@@ -21,7 +21,6 @@ package org.apache.samza.operators;
 import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.function.Function;
 
 import org.apache.samza.annotation.InterfaceStability;
 import org.apache.samza.operators.functions.FilterFunction;
@@ -237,34 +236,34 @@ public interface MessageStream<M> {
    * <p>
    * Unlike {@link #sendTo}, messages with a null key are all sent to partition 0.
    *
-   * @param keyExtractor the {@link Function} to extract the message and partition key from the input message.
+   * @param keyExtractor the {@link MapFunction} to extract the message and partition key from the input message.
    *                     Messages with a null key are all sent to partition 0.
-   * @param valueExtractor the {@link Function} to extract the value from the input message
+   * @param valueExtractor the {@link MapFunction} to extract the value from the input message
    * @param serde the {@link KVSerde} to use for (de)serializing the key and value.
    * @param id the unique id of this operator in this application
    * @param <K> the type of output key
    * @param <V> the type of output value
    * @return the repartitioned {@link MessageStream}
    */
-  <K, V> MessageStream<KV<K, V>> partitionBy(Function<? super M, ? extends K> keyExtractor,
-      Function<? super M, ? extends V> valueExtractor, KVSerde<K, V> serde, String id);
+  <K, V> MessageStream<KV<K, V>> partitionBy(MapFunction<? super M, ? extends K> keyExtractor,
+      MapFunction<? super M, ? extends V> valueExtractor, KVSerde<K, V> serde, String id);
 
   /**
-   * Same as calling {@link #partitionBy(Function, Function, KVSerde, String)} with a null KVSerde.
+   * Same as calling {@link #partitionBy(MapFunction, MapFunction, KVSerde, String)} with a null KVSerde.
    * <p>
    * Uses the default serde provided via {@link StreamGraph#setDefaultSerde}, which must be a KVSerde. If the default
    * serde is not a {@link KVSerde}, a runtime exception will be thrown. If no default serde has been provided
    * <b>before</b> calling this method, a {@code KVSerde<NoOpSerde, NoOpSerde>} is used.
    *
-   * @param keyExtractor the {@link Function} to extract the message and partition key from the input message
-   * @param valueExtractor the {@link Function} to extract the value from the input message
+   * @param keyExtractor the {@link MapFunction} to extract the message and partition key from the input message
+   * @param valueExtractor the {@link MapFunction} to extract the value from the input message
    * @param id the unique id of this operator in this application
    * @param <K> the type of output key
    * @param <V> the type of output value
    * @return the repartitioned {@link MessageStream}
    */
-  <K, V> MessageStream<KV<K, V>> partitionBy(Function<? super M, ? extends K> keyExtractor,
-      Function<? super M, ? extends V> valueExtractor, String id);
+  <K, V> MessageStream<KV<K, V>> partitionBy(MapFunction<? super M, ? extends K> keyExtractor,
+      MapFunction<? super M, ? extends V> valueExtractor, String id);
 
   /**
    * Sends messages in this {@link MessageStream} to a {@link Table}. The type of input message is expected

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java
index ea83ba4..faf9fc5 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/ClosableFunction.java
@@ -33,5 +33,8 @@ import org.apache.samza.annotation.InterfaceStability;
  */
 @InterfaceStability.Unstable
 public interface ClosableFunction {
+  /**
+   * Frees any resource acquired by the operators in {@link InitableFunction}
+   */
   default void close() {}
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-api/src/main/java/org/apache/samza/operators/functions/FilterFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/FilterFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/FilterFunction.java
index 31bbbd8..ce68e0f 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/functions/FilterFunction.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/FilterFunction.java
@@ -18,6 +18,7 @@
  */
 package org.apache.samza.operators.functions;
 
+import java.io.Serializable;
 import org.apache.samza.annotation.InterfaceStability;
 
 
@@ -28,7 +29,7 @@ import org.apache.samza.annotation.InterfaceStability;
  */
 @InterfaceStability.Unstable
 @FunctionalInterface
-public interface FilterFunction<M> extends InitableFunction, ClosableFunction {
+public interface FilterFunction<M> extends InitableFunction, ClosableFunction, Serializable {
 
   /**
    * Returns a boolean indicating whether this message should be retained or filtered out.

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-api/src/main/java/org/apache/samza/operators/functions/FlatMapFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/FlatMapFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/FlatMapFunction.java
index 7e9253e..63d7061 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/functions/FlatMapFunction.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/FlatMapFunction.java
@@ -18,6 +18,7 @@
  */
 package org.apache.samza.operators.functions;
 
+import java.io.Serializable;
 import org.apache.samza.annotation.InterfaceStability;
 
 import java.util.Collection;
@@ -31,7 +32,7 @@ import java.util.Collection;
  */
 @InterfaceStability.Unstable
 @FunctionalInterface
-public interface FlatMapFunction<M, OM>  extends InitableFunction, ClosableFunction {
+public interface FlatMapFunction<M, OM>  extends InitableFunction, ClosableFunction, Serializable {
 
   /**
    * Transforms the provided message into a collection of 0 or more messages.

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-api/src/main/java/org/apache/samza/operators/functions/FoldLeftFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/FoldLeftFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/FoldLeftFunction.java
index 78250e3..d6ba205 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/functions/FoldLeftFunction.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/FoldLeftFunction.java
@@ -19,16 +19,22 @@
 
 package org.apache.samza.operators.functions;
 
+import java.io.Serializable;
+import org.apache.samza.annotation.InterfaceStability;
+
+
 /**
- * Incrementally updates the window value as messages are added to the window.
+ * Incrementally updates the aggregated value as messages are added. Main usage is in {@link org.apache.samza.operators.windows.Window} operator.
  */
-public interface FoldLeftFunction<M, WV> extends InitableFunction, ClosableFunction {
+@InterfaceStability.Unstable
+@FunctionalInterface
+public interface FoldLeftFunction<M, WV> extends InitableFunction, ClosableFunction, Serializable {
 
   /**
-   * Incrementally updates the window value as messages are added to the window.
+   * Incrementally updates the aggregated value as messages are added.
    *
-   * @param message the message being added to the window
-   * @param oldValue the previous value associated with the window
+   * @param message the message being added to the aggregated value
+   * @param oldValue the previous value
    * @return the new value
    */
   WV apply(M message, WV oldValue);

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-api/src/main/java/org/apache/samza/operators/functions/JoinFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/JoinFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/JoinFunction.java
index 954083d..94a998d 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/functions/JoinFunction.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/JoinFunction.java
@@ -18,6 +18,7 @@
  */
 package org.apache.samza.operators.functions;
 
+import java.io.Serializable;
 import org.apache.samza.annotation.InterfaceStability;
 
 
@@ -30,7 +31,7 @@ import org.apache.samza.annotation.InterfaceStability;
  * @param <RM>  type of the joined message
  */
 @InterfaceStability.Unstable
-public interface JoinFunction<K, M, JM, RM>  extends InitableFunction, ClosableFunction {
+public interface JoinFunction<K, M, JM, RM>  extends InitableFunction, ClosableFunction, Serializable {
 
   /**
    * Joins the provided messages and returns the joined message.

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-api/src/main/java/org/apache/samza/operators/functions/MapFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/MapFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/MapFunction.java
index a8c139f..fad9cf8 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/functions/MapFunction.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/MapFunction.java
@@ -18,6 +18,7 @@
  */
 package org.apache.samza.operators.functions;
 
+import java.io.Serializable;
 import org.apache.samza.annotation.InterfaceStability;
 
 
@@ -29,7 +30,7 @@ import org.apache.samza.annotation.InterfaceStability;
  */
 @InterfaceStability.Unstable
 @FunctionalInterface
-public interface MapFunction<M, OM>  extends InitableFunction, ClosableFunction {
+public interface MapFunction<M, OM>  extends InitableFunction, ClosableFunction, Serializable {
 
   /**
    * Transforms the provided message into another message.

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-api/src/main/java/org/apache/samza/operators/functions/SinkFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/SinkFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/SinkFunction.java
index e290d7d..2b44125 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/functions/SinkFunction.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/SinkFunction.java
@@ -18,6 +18,7 @@
  */
 package org.apache.samza.operators.functions;
 
+import java.io.Serializable;
 import org.apache.samza.annotation.InterfaceStability;
 import org.apache.samza.task.MessageCollector;
 import org.apache.samza.task.TaskCoordinator;
@@ -30,7 +31,7 @@ import org.apache.samza.task.TaskCoordinator;
  */
 @InterfaceStability.Unstable
 @FunctionalInterface
-public interface SinkFunction<M>  extends InitableFunction, ClosableFunction {
+public interface SinkFunction<M>  extends InitableFunction, ClosableFunction, Serializable {
 
   /**
    * Allows sending the provided message to an output {@link org.apache.samza.system.SystemStream} using

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-api/src/main/java/org/apache/samza/operators/functions/StreamTableJoinFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/StreamTableJoinFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/StreamTableJoinFunction.java
index 6afcf67..356e07f 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/functions/StreamTableJoinFunction.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/StreamTableJoinFunction.java
@@ -18,6 +18,7 @@
  */
 package org.apache.samza.operators.functions;
 
+import java.io.Serializable;
 import org.apache.samza.annotation.InterfaceStability;
 
 
@@ -30,7 +31,7 @@ import org.apache.samza.annotation.InterfaceStability;
  * @param <JM> type of join results
  */
 @InterfaceStability.Unstable
-public interface StreamTableJoinFunction<K, M, R, JM> extends InitableFunction, ClosableFunction {
+public interface StreamTableJoinFunction<K, M, R, JM> extends InitableFunction, ClosableFunction, Serializable {
 
   /**
    * Joins the provided messages and table record, returns the joined message.

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-api/src/main/java/org/apache/samza/operators/functions/SupplierFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/SupplierFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/SupplierFunction.java
new file mode 100644
index 0000000..155fb0e
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/SupplierFunction.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.operators.functions;
+
+import java.io.Serializable;
+import org.apache.samza.annotation.InterfaceStability;
+
+
+/**
+ * A supplier to return a new value at each invocation
+ */
+@InterfaceStability.Unstable
+@FunctionalInterface
+public interface SupplierFunction<T> extends InitableFunction, ClosableFunction, Serializable {
+
+  /**
+   * Returns a value of type T
+   *
+   * @return a value for type T
+   */
+  T get();
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-api/src/main/java/org/apache/samza/operators/triggers/AnyTrigger.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/triggers/AnyTrigger.java b/samza-api/src/main/java/org/apache/samza/operators/triggers/AnyTrigger.java
index f52b57b..6bdf406 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/triggers/AnyTrigger.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/triggers/AnyTrigger.java
@@ -18,20 +18,24 @@
 */
 package org.apache.samza.operators.triggers;
 
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
+
 /**
  * A {@link Trigger} fires as soon as any of its individual triggers has fired.
  */
 public class AnyTrigger<M> implements Trigger<M> {
 
-  private final List<Trigger<M>> triggers;
+  private final ArrayList<Trigger<M>> triggers;
 
   AnyTrigger(List<Trigger<M>> triggers) {
-    this.triggers = triggers;
+    this.triggers = new ArrayList<>();
+    this.triggers.addAll(triggers);
   }
 
   public List<Trigger<M>> getTriggers() {
-    return triggers;
+    return Collections.unmodifiableList(triggers);
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-api/src/main/java/org/apache/samza/operators/triggers/Trigger.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/triggers/Trigger.java b/samza-api/src/main/java/org/apache/samza/operators/triggers/Trigger.java
index be0a877..f224fa2 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/triggers/Trigger.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/triggers/Trigger.java
@@ -20,6 +20,7 @@
 package org.apache.samza.operators.triggers;
 
 
+import java.io.Serializable;
 import org.apache.samza.annotation.InterfaceStability;
 
 /**
@@ -30,6 +31,6 @@ import org.apache.samza.annotation.InterfaceStability;
  * @param <M> the type of the incoming message
  */
 @InterfaceStability.Unstable
-public interface Trigger<M> {
+public interface Trigger<M> extends Serializable {
 
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-api/src/main/java/org/apache/samza/operators/windows/Window.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/Window.java b/samza-api/src/main/java/org/apache/samza/operators/windows/Window.java
index 1c0fa53..7534fca 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/windows/Window.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/Window.java
@@ -18,6 +18,7 @@
  */
 package org.apache.samza.operators.windows;
 
+import java.io.Serializable;
 import org.apache.samza.annotation.InterfaceStability;
 import org.apache.samza.operators.triggers.Trigger;
 
@@ -70,7 +71,7 @@ import org.apache.samza.operators.triggers.Trigger;
  * @param <WV> the type of the value in the window
  */
 @InterfaceStability.Unstable
-public interface Window<M, K, WV> {
+public interface Window<M, K, WV> extends Serializable {
 
   /**
    * Set the early triggers for this {@link Window}.

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-api/src/main/java/org/apache/samza/operators/windows/Windows.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/Windows.java b/samza-api/src/main/java/org/apache/samza/operators/windows/Windows.java
index 50391ff..4805a0e 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/windows/Windows.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/Windows.java
@@ -21,6 +21,8 @@ package org.apache.samza.operators.windows;
 
 import org.apache.samza.annotation.InterfaceStability;
 import org.apache.samza.operators.functions.FoldLeftFunction;
+import org.apache.samza.operators.functions.MapFunction;
+import org.apache.samza.operators.functions.SupplierFunction;
 import org.apache.samza.operators.triggers.TimeTrigger;
 import org.apache.samza.operators.triggers.Trigger;
 import org.apache.samza.operators.triggers.Triggers;
@@ -30,8 +32,6 @@ import org.apache.samza.serializers.Serde;
 
 import java.time.Duration;
 import java.util.Collection;
-import java.util.function.Function;
-import java.util.function.Supplier;
 
 /**
  * APIs for creating different types of {@link Window}s.
@@ -84,7 +84,7 @@ import java.util.function.Supplier;
  * and triggers are fired and window panes are emitted per-key. It is possible to construct "keyed" variants
  * of the window types above.
  *
- * <p> The value for the window can be updated incrementally by providing an {@code initialValue} {@link Supplier}
+ * <p> The value for the window can be updated incrementally by providing an {@code initialValue} {@link SupplierFunction}
  * and an aggregating {@link FoldLeftFunction}. The initial value supplier is invoked every time a new window is
  * created. The aggregating function is invoked for each incoming message for the window. If these are not provided,
  * the emitted {@link WindowPane} will contain a collection of messages in the window.
@@ -105,8 +105,8 @@ public final class Windows {
    *
    * <pre> {@code
    *    MessageStream<UserClick> stream = ...;
-   *    Function<UserClick, String> keyFn = ...;
-   *    Supplier<Integer> initialValue = () -> 0;
+   *    MapFunction<UserClick, String> keyFn = ...;
+   *    SupplierFunction<Integer> initialValue = () -> 0;
    *    FoldLeftFunction<UserClick, Integer, Integer> maxAggregator = (m, c) -> Math.max(parseInt(m), c);
    *    MessageStream<WindowPane<String, Integer>> windowedStream = stream.window(
    *        Windows.keyedTumblingWindow(keyFn, Duration.ofSeconds(10), maxAggregator));
@@ -125,16 +125,15 @@ public final class Windows {
    * @param <K> the type of the key in the {@link Window}
    * @return the created {@link Window} function.
    */
-  public static <M, K, WV> Window<M, K, WV> keyedTumblingWindow(Function<? super M, ? extends K> keyFn, Duration interval,
-      Supplier<? extends WV> initialValue, FoldLeftFunction<? super M, WV> aggregator, Serde<K> keySerde,
+  public static <M, K, WV> Window<M, K, WV> keyedTumblingWindow(MapFunction<? super M, ? extends K> keyFn, Duration interval,
+      SupplierFunction<? extends WV> initialValue, FoldLeftFunction<? super M, WV> aggregator, Serde<K> keySerde,
       Serde<WV> windowValueSerde) {
 
     Trigger<M> defaultTrigger = new TimeTrigger<>(interval);
-    return new WindowInternal<>(defaultTrigger, (Supplier<WV>) initialValue, (FoldLeftFunction<M, WV>) aggregator,
-        (Function<M, K>) keyFn, null, WindowType.TUMBLING, keySerde, windowValueSerde, null);
+    return new WindowInternal<>(defaultTrigger, (SupplierFunction<WV>) initialValue, (FoldLeftFunction<M, WV>) aggregator,
+        (MapFunction<M, K>) keyFn, null, WindowType.TUMBLING, keySerde, windowValueSerde, null);
   }
 
-
   /**
    * Creates a {@link Window} that groups incoming messages into fixed-size, non-overlapping
    * processing time based windows using the provided keyFn.
@@ -157,12 +156,12 @@ public final class Windows {
    * @param <K> the type of the key in the {@link Window}
    * @return the created {@link Window} function
    */
-  public static <M, K> Window<M, K, Collection<M>> keyedTumblingWindow(Function<M, K> keyFn, Duration interval,
+  public static <M, K> Window<M, K, Collection<M>> keyedTumblingWindow(MapFunction<M, K> keyFn, Duration interval,
       Serde<K> keySerde, Serde<M> msgSerde) {
 
     Trigger<M> defaultTrigger = new TimeTrigger<>(interval);
-    return new WindowInternal<>(defaultTrigger, null, null, keyFn, null,
-        WindowType.TUMBLING, keySerde, null, msgSerde);
+    return new WindowInternal<>(defaultTrigger, null, null, keyFn, null, WindowType.TUMBLING,
+        keySerde, null, msgSerde);
   }
 
   /**
@@ -173,7 +172,7 @@ public final class Windows {
    *
    * <pre> {@code
    *    MessageStream<String> stream = ...;
-   *    Supplier<Integer> initialValue = () -> 0;
+   *    SupplierFunction<Integer> initialValue = () -> 0;
    *    FoldLeftFunction<String, Integer, Integer> maxAggregator = (m, c) -> Math.max(parseInt(m), c);
    *    MessageStream<WindowPane<Void, Integer>> windowedStream = stream.window(
    *        Windows.tumblingWindow(Duration.ofSeconds(10), maxAggregator));
@@ -189,10 +188,10 @@ public final class Windows {
    * @param <WV> the type of the {@link WindowPane} output value
    * @return the created {@link Window} function
    */
-  public static <M, WV> Window<M, Void, WV> tumblingWindow(Duration interval, Supplier<? extends WV> initialValue,
+  public static <M, WV> Window<M, Void, WV> tumblingWindow(Duration interval, SupplierFunction<? extends WV> initialValue,
       FoldLeftFunction<? super M, WV> aggregator, Serde<WV> windowValueSerde) {
     Trigger<M> defaultTrigger = new TimeTrigger<>(interval);
-    return new WindowInternal<>(defaultTrigger, (Supplier<WV>) initialValue, (FoldLeftFunction<M, WV>) aggregator,
+    return new WindowInternal<>(defaultTrigger, (SupplierFunction<WV>) initialValue, (FoldLeftFunction<M, WV>) aggregator,
         null, null, WindowType.TUMBLING, null, windowValueSerde, null);
   }
 
@@ -221,9 +220,8 @@ public final class Windows {
    */
   public static <M> Window<M, Void, Collection<M>> tumblingWindow(Duration duration, Serde<M> msgSerde) {
     Trigger<M> defaultTrigger = new TimeTrigger<>(duration);
-
-    return new WindowInternal<>(defaultTrigger, null, null, null,
-       null, WindowType.TUMBLING, null, null, msgSerde);
+    return new WindowInternal<>(defaultTrigger, null, null, null, null,
+        WindowType.TUMBLING, null, null, msgSerde);
   }
 
   /**
@@ -238,7 +236,7 @@ public final class Windows {
    *
    * <pre> {@code
    *    MessageStream<UserClick> stream = ...;
-   *    Supplier<Integer> initialValue = () -> 0;
+   *    SupplierFunction<Integer> initialValue = () -> 0;
    *    FoldLeftFunction<UserClick, Integer, Integer> maxAggregator = (m, c) -> Math.max(parseInt(m), c);
    *    Function<UserClick, String> userIdExtractor = m -> m.getUserId()..;
    *    MessageStream<WindowPane<String, Integer>> windowedStream = stream.window(
@@ -258,12 +256,12 @@ public final class Windows {
    * @param <WV> the type of the output value in the {@link WindowPane}
    * @return the created {@link Window} function
    */
-  public static <M, K, WV> Window<M, K, WV> keyedSessionWindow(Function<? super M, ? extends K> keyFn,
-      Duration sessionGap, Supplier<? extends WV> initialValue, FoldLeftFunction<? super M, WV> aggregator,
+  public static <M, K, WV> Window<M, K, WV> keyedSessionWindow(MapFunction<? super M, ? extends K> keyFn,
+      Duration sessionGap, SupplierFunction<? extends WV> initialValue, FoldLeftFunction<? super M, WV> aggregator,
       Serde<K> keySerde, Serde<WV> windowValueSerde) {
     Trigger<M> defaultTrigger = Triggers.timeSinceLastMessage(sessionGap);
-    return new WindowInternal<>(defaultTrigger, (Supplier<WV>) initialValue, (FoldLeftFunction<M, WV>) aggregator,
-        (Function<M, K>) keyFn, null, WindowType.SESSION, keySerde, windowValueSerde, null);
+    return new WindowInternal<>(defaultTrigger, (SupplierFunction<WV>) initialValue, (FoldLeftFunction<M, WV>) aggregator,
+        (MapFunction<M, K>) keyFn, null, WindowType.SESSION, keySerde, windowValueSerde, null);
   }
 
   /**
@@ -278,7 +276,7 @@ public final class Windows {
    *
    * <pre> {@code
    *    MessageStream<UserClick> stream = ...;
-   *    Supplier<Integer> initialValue = () -> 0;
+   *    SupplierFunction<Integer> initialValue = () -> 0;
    *    FoldLeftFunction<UserClick, Integer, Integer> maxAggregator = (m, c)-> Math.max(parseIntField(m), c);
    *    Function<UserClick, String> userIdExtractor = m -> m.getUserId()..;
    *    MessageStream<WindowPane<String>, Collection<M>> windowedStream = stream.window(
@@ -294,11 +292,10 @@ public final class Windows {
    * @param <K> the type of the key in the {@link Window}
    * @return the created {@link Window} function
    */
-  public static <M, K> Window<M, K, Collection<M>> keyedSessionWindow(Function<? super M, ? extends K> keyFn,
+  public static <M, K> Window<M, K, Collection<M>> keyedSessionWindow(MapFunction<? super M, ? extends K> keyFn,
       Duration sessionGap, Serde<K> keySerde, Serde<M> msgSerde) {
-
     Trigger<M> defaultTrigger = Triggers.timeSinceLastMessage(sessionGap);
-    return new WindowInternal<>(defaultTrigger, null, null, (Function<M, K>) keyFn,
+    return new WindowInternal<>(defaultTrigger, null, null, (MapFunction<M, K>) keyFn,
         null, WindowType.SESSION, keySerde, null, msgSerde);
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-api/src/main/java/org/apache/samza/operators/windows/internal/WindowInternal.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/internal/WindowInternal.java b/samza-api/src/main/java/org/apache/samza/operators/windows/internal/WindowInternal.java
index bc71872..ff19aba 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/windows/internal/WindowInternal.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/internal/WindowInternal.java
@@ -18,14 +18,14 @@
  */
 package org.apache.samza.operators.windows.internal;
 import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.operators.functions.MapFunction;
+import org.apache.samza.operators.functions.SupplierFunction;
 import org.apache.samza.operators.functions.FoldLeftFunction;
 import org.apache.samza.operators.triggers.Trigger;
 import org.apache.samza.operators.windows.AccumulationMode;
 import org.apache.samza.operators.windows.Window;
 import org.apache.samza.serializers.Serde;
 
-import java.util.function.Function;
-import java.util.function.Supplier;
 
 /**
  *  Internal representation of a {@link Window}. This specifies default, early and late triggers for the {@link Window}
@@ -45,7 +45,7 @@ public final class WindowInternal<M, WK, WV> implements Window<M, WK, WV> {
   /**
    * The supplier of initial value to be used for windowed aggregations
    */
-  private final Supplier<WV> initializer;
+  private final SupplierFunction<WV> initializer;
 
   /*
    * The function that is applied each time a {@link MessageEnvelope} is added to this window.
@@ -55,28 +55,32 @@ public final class WindowInternal<M, WK, WV> implements Window<M, WK, WV> {
   /*
    * The function that extracts the key from a {@link MessageEnvelope}
    */
-  private final Function<M, WK> keyExtractor;
+  private final MapFunction<M, WK> keyExtractor;
 
   /*
    * The function that extracts the event time from a {@link MessageEnvelope}
    */
-  private final Function<M, Long> eventTimeExtractor;
+  private final MapFunction<M, Long> eventTimeExtractor;
 
   /**
    * The type of this window. Tumbling and Session windows are supported for now.
    */
   private final WindowType windowType;
 
-  private final Serde<WK> keySerde;
-  private final Serde<WV> windowValSerde;
-  private final Serde<M> msgSerde;
-
   private Trigger<M> earlyTrigger;
   private Trigger<M> lateTrigger;
   private AccumulationMode mode;
 
-  public WindowInternal(Trigger<M> defaultTrigger, Supplier<WV> initializer, FoldLeftFunction<M, WV> foldLeftFunction,
-      Function<M, WK> keyExtractor, Function<M, Long> eventTimeExtractor, WindowType windowType, Serde<WK> keySerde,
+  /**
+   * The following {@link Serde}s are serialized by the ExecutionPlanner when generating the store configs, and deserialized
+   * once during startup in SamzaContainer. They don't need to be deserialized here on a per-task basis
+   */
+  private transient final Serde<WK> keySerde;
+  private transient final Serde<WV> windowValSerde;
+  private transient final Serde<M> msgSerde;
+
+  public WindowInternal(Trigger<M> defaultTrigger, SupplierFunction<WV> initializer, FoldLeftFunction<M, WV> foldLeftFunction,
+      MapFunction<M, WK> keyExtractor, MapFunction<M, Long> eventTimeExtractor, WindowType windowType, Serde<WK> keySerde,
       Serde<WV> windowValueSerde, Serde<M> msgSerde) {
     this.defaultTrigger = defaultTrigger;
     this.initializer = initializer;
@@ -121,7 +125,7 @@ public final class WindowInternal<M, WK, WV> implements Window<M, WK, WV> {
     return lateTrigger;
   }
 
-  public Supplier<WV> getInitializer() {
+  public SupplierFunction<WV> getInitializer() {
     return initializer;
   }
 
@@ -129,11 +133,11 @@ public final class WindowInternal<M, WK, WV> implements Window<M, WK, WV> {
     return foldLeftFunction;
   }
 
-  public Function<M, WK> getKeyExtractor() {
+  public MapFunction<M, WK> getKeyExtractor() {
     return keyExtractor;
   }
 
-  public Function<M, Long> getEventTimeExtractor() {
+  public MapFunction<M, Long> getEventTimeExtractor() {
     return eventTimeExtractor;
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-api/src/main/java/org/apache/samza/serializers/SerializableSerde.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/serializers/SerializableSerde.java b/samza-api/src/main/java/org/apache/samza/serializers/SerializableSerde.java
index d70746c..d49518c 100644
--- a/samza-api/src/main/java/org/apache/samza/serializers/SerializableSerde.java
+++ b/samza-api/src/main/java/org/apache/samza/serializers/SerializableSerde.java
@@ -68,7 +68,7 @@ public class SerializableSerde<T extends Serializable> implements Serde<T> {
         ois = new ObjectInputStream(bis);
         return (T) ois.readObject();
       } catch (IOException | ClassNotFoundException e) {
-        throw new SamzaException("Error reading from input stream.");
+        throw new SamzaException("Error reading from input stream.", e);
       } finally {
         try {
           if (ois != null) {

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 ce67d8d..cd86426 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
@@ -19,6 +19,7 @@
 
 package org.apache.samza.system;
 
+import java.io.Serializable;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
@@ -33,7 +34,7 @@ import java.util.Map;
  *
  * It is immutable by design.
  */
-public class StreamSpec {
+public class StreamSpec implements Serializable {
 
   private static final int DEFAULT_PARTITION_COUNT = 1;
 

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-api/src/main/java/org/apache/samza/system/SystemStreamPartition.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/SystemStreamPartition.java b/samza-api/src/main/java/org/apache/samza/system/SystemStreamPartition.java
index 95cc266..e9ca9f7 100644
--- a/samza-api/src/main/java/org/apache/samza/system/SystemStreamPartition.java
+++ b/samza-api/src/main/java/org/apache/samza/system/SystemStreamPartition.java
@@ -60,7 +60,7 @@ public class SystemStreamPartition extends SystemStream implements Comparable<Sy
   public Partition getPartition() {
     return partition;
   }
-  
+
   public SystemStream getSystemStream() {
     return new SystemStream(system, stream);
   }
@@ -69,7 +69,7 @@ public class SystemStreamPartition extends SystemStream implements Comparable<Sy
   public int hashCode() {
     return hash;
   }
-  
+
   private int computeHashCode() {
     final int prime = 31;
     int result = super.hashCode();

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-api/src/main/java/org/apache/samza/table/TableSpec.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/table/TableSpec.java b/samza-api/src/main/java/org/apache/samza/table/TableSpec.java
index 68043f9..ba57c2f 100644
--- a/samza-api/src/main/java/org/apache/samza/table/TableSpec.java
+++ b/samza-api/src/main/java/org/apache/samza/table/TableSpec.java
@@ -18,6 +18,7 @@
  */
 package org.apache.samza.table;
 
+import java.io.Serializable;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
@@ -41,12 +42,17 @@ import org.apache.samza.serializers.KVSerde;
  * It is immutable by design.
  */
 @InterfaceStability.Unstable
-public class TableSpec {
+public class TableSpec implements Serializable {
 
   private final String id;
-  private final KVSerde serde;
   private final String tableProviderFactoryClassName;
-  private final Map<String, String> config = new HashMap<>();
+
+  /**
+   * The following fields are serialized by the ExecutionPlanner when generating the configs for a table, and deserialized
+   * once during startup in SamzaContainer. They don't need to be deserialized here on a per-task basis
+   */
+  private transient final KVSerde serde;
+  private transient final Map<String, String> config = new HashMap<>();
 
   /**
    * Default constructor

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowPane.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowPane.java b/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowPane.java
index 4184c9d..19cce6f 100644
--- a/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowPane.java
+++ b/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowPane.java
@@ -27,7 +27,7 @@ import static org.junit.Assert.assertEquals;
 public class TestWindowPane {
   @Test
   public void testConstructor() {
-    WindowPane<String, Integer> wndOutput = new WindowPane(new WindowKey("testMsg", null), 10, AccumulationMode.DISCARDING, FiringType.EARLY);
+    WindowPane<String, Integer> wndOutput = new WindowPane<>(new WindowKey<>("testMsg", null), 10, AccumulationMode.DISCARDING, FiringType.EARLY);
     assertEquals(wndOutput.getKey().getKey(), "testMsg");
     assertEquals(wndOutput.getMessage(), Integer.valueOf(10));
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 e2c122a..9d8bd5f 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
@@ -34,7 +34,7 @@ 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;
+import org.apache.samza.operators.OperatorSpecGraph;
 import org.apache.samza.operators.spec.JoinOperatorSpec;
 import org.apache.samza.operators.spec.OperatorSpec;
 import org.apache.samza.system.StreamSpec;
@@ -61,18 +61,18 @@ public class ExecutionPlanner {
     this.streamManager = streamManager;
   }
 
-  public ExecutionPlan plan(StreamGraphImpl streamGraph) throws Exception {
+  public ExecutionPlan plan(OperatorSpecGraph specGraph) throws Exception {
     validateConfig();
 
     // create physical job graph based on stream graph
-    JobGraph jobGraph = createJobGraph(streamGraph);
+    JobGraph jobGraph = createJobGraph(specGraph);
 
     // fetch the external streams partition info
     updateExistingPartitions(jobGraph, streamManager);
 
     if (!jobGraph.getIntermediateStreamEdges().isEmpty()) {
       // figure out the partitions for internal streams
-      calculatePartitions(streamGraph, jobGraph);
+      calculatePartitions(jobGraph);
     }
 
     return jobGraph;
@@ -91,12 +91,12 @@ public class ExecutionPlanner {
   /**
    * Create the physical graph from StreamGraph
    */
-  /* package private */ JobGraph createJobGraph(StreamGraphImpl streamGraph) {
-    JobGraph jobGraph = new JobGraph(config);
-    Set<StreamSpec> sourceStreams = new HashSet<>(streamGraph.getInputOperators().keySet());
-    Set<StreamSpec> sinkStreams = new HashSet<>(streamGraph.getOutputStreams().keySet());
+  /* package private */ JobGraph createJobGraph(OperatorSpecGraph specGraph) {
+    JobGraph jobGraph = new JobGraph(config, specGraph);
+    Set<StreamSpec> sourceStreams = new HashSet<>(specGraph.getInputOperators().keySet());
+    Set<StreamSpec> sinkStreams = new HashSet<>(specGraph.getOutputStreams().keySet());
     Set<StreamSpec> intStreams = new HashSet<>(sourceStreams);
-    Set<TableSpec> tables = new HashSet<>(streamGraph.getTables().keySet());
+    Set<TableSpec> tables = new HashSet<>(specGraph.getTables().keySet());
     intStreams.retainAll(sinkStreams);
     sourceStreams.removeAll(intStreams);
     sinkStreams.removeAll(intStreams);
@@ -104,7 +104,7 @@ public class ExecutionPlanner {
     // For this phase, we have a single job node for the whole dag
     String jobName = config.get(JobConfig.JOB_NAME());
     String jobId = config.get(JobConfig.JOB_ID(), "1");
-    JobNode node = jobGraph.getOrCreateJobNode(jobName, jobId, streamGraph);
+    JobNode node = jobGraph.getOrCreateJobNode(jobName, jobId);
 
     // add sources
     sourceStreams.forEach(spec -> jobGraph.addSource(spec, node));
@@ -126,9 +126,9 @@ public class ExecutionPlanner {
   /**
    * Figure out the number of partitions of all streams
    */
-  /* package private */ void calculatePartitions(StreamGraphImpl streamGraph, JobGraph jobGraph) {
+  /* package private */ void calculatePartitions(JobGraph jobGraph) {
     // calculate the partitions for the input streams of join operators
-    calculateJoinInputPartitions(streamGraph, jobGraph);
+    calculateJoinInputPartitions(jobGraph);
 
     // calculate the partitions for the rest of intermediate streams
     calculateIntStreamPartitions(jobGraph, config);
@@ -172,7 +172,7 @@ public class ExecutionPlanner {
   /**
    * Calculate the partitions for the input streams of join operators
    */
-  /* package private */ static void calculateJoinInputPartitions(StreamGraphImpl streamGraph, JobGraph jobGraph) {
+  /* package private */ static void calculateJoinInputPartitions(JobGraph jobGraph) {
     // mapping from a source stream to all join specs reachable from it
     Multimap<OperatorSpec, StreamEdge> joinSpecToStreamEdges = HashMultimap.create();
     // reverse mapping of the above
@@ -182,7 +182,7 @@ public class ExecutionPlanner {
     // The visited set keeps track of the join specs that have been already inserted in the queue before
     Set<OperatorSpec> visited = new HashSet<>();
 
-    streamGraph.getInputOperators().entrySet().forEach(entry -> {
+    jobGraph.getSpecGraph().getInputOperators().entrySet().forEach(entry -> {
         StreamEdge streamEdge = jobGraph.getOrCreateStreamEdge(entry.getKey());
         // Traverses the StreamGraph to find and update mappings for all Joins reachable from this input StreamEdge
         findReachableJoins(entry.getValue(), streamEdge, joinSpecToStreamEdges, streamEdgeToJoinSpecs,

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 abd3ce7..843db85 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
@@ -34,7 +34,7 @@ import java.util.stream.Collectors;
 import org.apache.samza.config.ApplicationConfig;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JobConfig;
-import org.apache.samza.operators.StreamGraphImpl;
+import org.apache.samza.operators.OperatorSpecGraph;
 import org.apache.samza.system.StreamSpec;
 import org.apache.samza.table.TableSpec;
 import org.slf4j.Logger;
@@ -60,13 +60,15 @@ import org.slf4j.LoggerFactory;
   private final Set<TableSpec> tables = new HashSet<>();
   private final Config config;
   private final JobGraphJsonGenerator jsonGenerator = new JobGraphJsonGenerator();
+  private final OperatorSpecGraph specGraph;
 
   /**
    * The JobGraph is only constructed by the {@link ExecutionPlanner}.
    * @param config Config
    */
-  JobGraph(Config config) {
+  JobGraph(Config config, OperatorSpecGraph specGraph) {
     this.config = config;
+    this.specGraph = specGraph;
   }
 
   @Override
@@ -107,6 +109,10 @@ import org.slf4j.LoggerFactory;
     return new ApplicationConfig(config);
   }
 
+  public OperatorSpecGraph getSpecGraph() {
+    return specGraph;
+  }
+
   /**
    * Add a source stream to a {@link JobNode}
    * @param input source stream
@@ -152,11 +158,11 @@ import org.slf4j.LoggerFactory;
    * @param jobId id of the job
    * @return
    */
-  JobNode getOrCreateJobNode(String jobName, String jobId, StreamGraphImpl streamGraph) {
+  JobNode getOrCreateJobNode(String jobName, String jobId) {
     String nodeId = JobNode.createId(jobName, jobId);
     JobNode node = nodes.get(nodeId);
     if (node == null) {
-      node = new JobNode(jobName, jobId, streamGraph, config);
+      node = new JobNode(jobName, jobId, specGraph, config);
       nodes.put(nodeId, node);
     }
     return node;

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java b/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java
index 48d2219..298042b 100644
--- a/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java
+++ b/samza-core/src/main/java/org/apache/samza/execution/JobGraphJsonGenerator.java
@@ -170,7 +170,7 @@ import org.codehaus.jackson.map.ObjectMapper;
   private OperatorGraphJson buildOperatorGraphJson(JobNode jobNode) {
     OperatorGraphJson opGraph = new OperatorGraphJson();
     opGraph.inputStreams = new ArrayList<>();
-    jobNode.getStreamGraph().getInputOperators().forEach((streamSpec, operatorSpec) -> {
+    jobNode.getSpecGraph().getInputOperators().forEach((streamSpec, operatorSpec) -> {
         StreamJson inputJson = new StreamJson();
         opGraph.inputStreams.add(inputJson);
         inputJson.streamId = streamSpec.getId();
@@ -181,7 +181,7 @@ import org.codehaus.jackson.map.ObjectMapper;
       });
 
     opGraph.outputStreams = new ArrayList<>();
-    jobNode.getStreamGraph().getOutputStreams().keySet().forEach(streamSpec -> {
+    jobNode.getSpecGraph().getOutputStreams().keySet().forEach(streamSpec -> {
         StreamJson outputJson = new StreamJson();
         outputJson.streamId = streamSpec.getId();
         opGraph.outputStreams.add(outputJson);

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 8abd463..db44d9f 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
@@ -39,7 +39,7 @@ import org.apache.samza.config.StorageConfig;
 import org.apache.samza.config.StreamConfig;
 import org.apache.samza.config.TaskConfig;
 import org.apache.samza.config.TaskConfigJava;
-import org.apache.samza.operators.StreamGraphImpl;
+import org.apache.samza.operators.OperatorSpecGraph;
 import org.apache.samza.operators.spec.InputOperatorSpec;
 import org.apache.samza.operators.spec.JoinOperatorSpec;
 import org.apache.samza.operators.spec.OperatorSpec;
@@ -73,22 +73,22 @@ public class JobNode {
   private final String jobName;
   private final String jobId;
   private final String id;
-  private final StreamGraphImpl streamGraph;
+  private final OperatorSpecGraph specGraph;
   private final List<StreamEdge> inEdges = new ArrayList<>();
   private final List<StreamEdge> outEdges = new ArrayList<>();
   private final List<TableSpec> tables = new ArrayList<>();
   private final Config config;
 
-  JobNode(String jobName, String jobId, StreamGraphImpl streamGraph, Config config) {
+  JobNode(String jobName, String jobId, OperatorSpecGraph specGraph, Config config) {
     this.jobName = jobName;
     this.jobId = jobId;
     this.id = createId(jobName, jobId);
-    this.streamGraph = streamGraph;
+    this.specGraph = specGraph;
     this.config = config;
   }
 
-  public StreamGraphImpl getStreamGraph() {
-    return streamGraph;
+  public OperatorSpecGraph getSpecGraph() {
+    return this.specGraph;
   }
 
   public  String getId() {
@@ -154,7 +154,7 @@ public class JobNode {
     }
 
     // set triggering interval if a window or join is defined
-    if (streamGraph.hasWindowOrJoins()) {
+    if (specGraph.hasWindowOrJoins()) {
       if ("-1".equals(config.get(TaskConfig.WINDOW_MS(), "-1"))) {
         long triggerInterval = computeTriggerInterval();
         log.info("Using triggering interval: {} for jobName: {}", triggerInterval, jobName);
@@ -163,7 +163,7 @@ public class JobNode {
       }
     }
 
-    streamGraph.getAllOperatorSpecs().forEach(opSpec -> {
+    specGraph.getAllOperatorSpecs().forEach(opSpec -> {
         if (opSpec instanceof StatefulOperatorSpec) {
           ((StatefulOperatorSpec) opSpec).getStoreDescriptors()
               .forEach(sd -> configs.putAll(sd.getStorageConfigs()));
@@ -228,14 +228,14 @@ public class JobNode {
     // collect all key and msg serde instances for streams
     Map<String, Serde> streamKeySerdes = new HashMap<>();
     Map<String, Serde> streamMsgSerdes = new HashMap<>();
-    Map<StreamSpec, InputOperatorSpec> inputOperators = streamGraph.getInputOperators();
+    Map<StreamSpec, InputOperatorSpec> inputOperators = specGraph.getInputOperators();
     inEdges.forEach(edge -> {
         String streamId = edge.getStreamSpec().getId();
         InputOperatorSpec inputOperatorSpec = inputOperators.get(edge.getStreamSpec());
         streamKeySerdes.put(streamId, inputOperatorSpec.getKeySerde());
         streamMsgSerdes.put(streamId, inputOperatorSpec.getValueSerde());
       });
-    Map<StreamSpec, OutputStreamImpl> outputStreams = streamGraph.getOutputStreams();
+    Map<StreamSpec, OutputStreamImpl> outputStreams = specGraph.getOutputStreams();
     outEdges.forEach(edge -> {
         String streamId = edge.getStreamSpec().getId();
         OutputStreamImpl outputStream = outputStreams.get(edge.getStreamSpec());
@@ -246,7 +246,7 @@ public class JobNode {
     // collect all key and msg serde instances for stores
     Map<String, Serde> storeKeySerdes = new HashMap<>();
     Map<String, Serde> storeMsgSerdes = new HashMap<>();
-    streamGraph.getAllOperatorSpecs().forEach(opSpec -> {
+    specGraph.getAllOperatorSpecs().forEach(opSpec -> {
         if (opSpec instanceof StatefulOperatorSpec) {
           ((StatefulOperatorSpec) opSpec).getStoreDescriptors().forEach(storeDescriptor -> {
               storeKeySerdes.put(storeDescriptor.getStoreName(), storeDescriptor.getKeySerde());
@@ -320,8 +320,8 @@ public class JobNode {
    * Computes the triggering interval to use during the execution of this {@link JobNode}
    */
   private long computeTriggerInterval() {
-    // Obtain the operator specs from the streamGraph
-    Collection<OperatorSpec> operatorSpecs = streamGraph.getAllOperatorSpecs();
+    // Obtain the operator specs from the specGraph
+    Collection<OperatorSpec> operatorSpecs = specGraph.getAllOperatorSpecs();
 
     // Filter out window operators, and obtain a list of their triggering interval values
     List<Long> windowTimerIntervals = operatorSpecs.stream()

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java b/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
index 1681f30..6922c76 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
@@ -21,7 +21,6 @@ package org.apache.samza.operators;
 
 import java.time.Duration;
 import java.util.Collection;
-import java.util.function.Function;
 
 import org.apache.samza.SamzaException;
 import org.apache.samza.operators.functions.FilterFunction;
@@ -64,16 +63,16 @@ import org.apache.samza.table.TableSpec;
  */
 public class MessageStreamImpl<M> implements MessageStream<M> {
   /**
-   * The {@link StreamGraphImpl} that contains this {@link MessageStreamImpl}
+   * The {@link StreamGraphSpec} that contains this {@link MessageStreamImpl}
    */
-  private final StreamGraphImpl graph;
+  private final StreamGraphSpec graph;
 
   /**
    * The {@link OperatorSpec} associated with this {@link MessageStreamImpl}
    */
   private final OperatorSpec operatorSpec;
 
-  public MessageStreamImpl(StreamGraphImpl graph, OperatorSpec<?, M> operatorSpec) {
+  public MessageStreamImpl(StreamGraphSpec graph, OperatorSpec<?, M> operatorSpec) {
     this.graph = graph;
     this.operatorSpec = operatorSpec;
   }
@@ -81,7 +80,7 @@ public class MessageStreamImpl<M> implements MessageStream<M> {
   @Override
   public <TM> MessageStream<TM> map(MapFunction<? super M, ? extends TM> mapFn) {
     String opId = this.graph.getNextOpId(OpCode.MAP);
-    OperatorSpec<M, TM> op = OperatorSpecs.createMapOperatorSpec(mapFn, opId);
+    StreamOperatorSpec<M, TM> op = OperatorSpecs.createMapOperatorSpec(mapFn, opId);
     this.operatorSpec.registerNextOperatorSpec(op);
     return new MessageStreamImpl<>(this.graph, op);
   }
@@ -89,7 +88,7 @@ public class MessageStreamImpl<M> implements MessageStream<M> {
   @Override
   public MessageStream<M> filter(FilterFunction<? super M> filterFn) {
     String opId = this.graph.getNextOpId(OpCode.FILTER);
-    OperatorSpec<M, M> op = OperatorSpecs.createFilterOperatorSpec(filterFn, opId);
+    StreamOperatorSpec<M, M> op = OperatorSpecs.createFilterOperatorSpec(filterFn, opId);
     this.operatorSpec.registerNextOperatorSpec(op);
     return new MessageStreamImpl<>(this.graph, op);
   }
@@ -97,7 +96,7 @@ public class MessageStreamImpl<M> implements MessageStream<M> {
   @Override
   public <TM> MessageStream<TM> flatMap(FlatMapFunction<? super M, ? extends TM> flatMapFn) {
     String opId = this.graph.getNextOpId(OpCode.FLAT_MAP);
-    OperatorSpec<M, TM> op = OperatorSpecs.createFlatMapOperatorSpec(flatMapFn, opId);
+    StreamOperatorSpec<M, TM> op = OperatorSpecs.createFlatMapOperatorSpec(flatMapFn, opId);
     this.operatorSpec.registerNextOperatorSpec(op);
     return new MessageStreamImpl<>(this.graph, op);
   }
@@ -112,15 +111,15 @@ public class MessageStreamImpl<M> implements MessageStream<M> {
   @Override
   public void sendTo(OutputStream<M> outputStream) {
     String opId = this.graph.getNextOpId(OpCode.SEND_TO);
-    OutputOperatorSpec<M> op = OperatorSpecs.createSendToOperatorSpec((OutputStreamImpl<M>) outputStream, opId);
+    OutputOperatorSpec<M> op = OperatorSpecs.createSendToOperatorSpec(
+        (OutputStreamImpl<M>) outputStream, opId);
     this.operatorSpec.registerNextOperatorSpec(op);
   }
 
   @Override
   public <K, WV> MessageStream<WindowPane<K, WV>> window(Window<M, K, WV> window, String userDefinedId) {
     String opId = this.graph.getNextOpId(OpCode.WINDOW, userDefinedId);
-    OperatorSpec<M, WindowPane<K, WV>> op = OperatorSpecs.createWindowOperatorSpec(
-        (WindowInternal<M, K, WV>) window, opId);
+    OperatorSpec<M, WindowPane<K, WV>> op = OperatorSpecs.createWindowOperatorSpec((WindowInternal<M, K, WV>) window, opId);
     this.operatorSpec.registerNextOperatorSpec(op);
     return new MessageStreamImpl<>(this.graph, op);
   }
@@ -131,24 +130,24 @@ public class MessageStreamImpl<M> implements MessageStream<M> {
       Serde<K> keySerde, Serde<M> messageSerde, Serde<OM> otherMessageSerde,
       Duration ttl, String userDefinedId) {
     if (otherStream.equals(this)) throw new SamzaException("Cannot join a MessageStream with itself.");
-    OperatorSpec<?, OM> otherOpSpec = ((MessageStreamImpl<OM>) otherStream).getOperatorSpec();
     String opId = this.graph.getNextOpId(OpCode.JOIN, userDefinedId);
-    JoinOperatorSpec<K, M, OM, JM> joinOpSpec =
-        OperatorSpecs.createJoinOperatorSpec(this.operatorSpec, otherOpSpec, (JoinFunction<K, M, OM, JM>) joinFn,
-            keySerde, messageSerde, otherMessageSerde, ttl.toMillis(), opId);
-
-    this.operatorSpec.registerNextOperatorSpec(joinOpSpec);
-    otherOpSpec.registerNextOperatorSpec((OperatorSpec<OM, ?>) joinOpSpec);
+    OperatorSpec<?, OM> otherOpSpec = ((MessageStreamImpl<OM>) otherStream).getOperatorSpec();
+    JoinOperatorSpec<K, M, OM, JM> op =
+        OperatorSpecs.createJoinOperatorSpec(this.operatorSpec, otherOpSpec, (JoinFunction<K, M, OM, JM>) joinFn, keySerde,
+            messageSerde, otherMessageSerde, ttl.toMillis(), opId);
+    this.operatorSpec.registerNextOperatorSpec(op);
+    otherOpSpec.registerNextOperatorSpec((OperatorSpec<OM, ?>) op);
 
-    return new MessageStreamImpl<>(this.graph, joinOpSpec);
+    return new MessageStreamImpl<>(this.graph, op);
   }
 
   @Override
   public <K, R extends KV, JM> MessageStream<JM> join(Table<R> table,
       StreamTableJoinFunction<? extends K, ? super M, ? super R, ? extends JM> joinFn) {
+    String opId = this.graph.getNextOpId(OpCode.JOIN);
     TableSpec tableSpec = ((TableImpl) table).getTableSpec();
     StreamTableJoinOperatorSpec<K, M, R, JM> joinOpSpec = OperatorSpecs.createStreamTableJoinOperatorSpec(
-        tableSpec, (StreamTableJoinFunction<K, M, R, JM>) joinFn, this.graph.getNextOpId(OpCode.JOIN));
+        tableSpec, (StreamTableJoinFunction<K, M, R, JM>) joinFn, opId);
     this.operatorSpec.registerNextOperatorSpec(joinOpSpec);
     return new MessageStreamImpl<>(this.graph, joinOpSpec);
   }
@@ -157,46 +156,38 @@ public class MessageStreamImpl<M> implements MessageStream<M> {
   public MessageStream<M> merge(Collection<? extends MessageStream<? extends M>> otherStreams) {
     if (otherStreams.isEmpty()) return this;
     String opId = this.graph.getNextOpId(OpCode.MERGE);
-    StreamOperatorSpec<M, M> opSpec = OperatorSpecs.createMergeOperatorSpec(opId);
-    this.operatorSpec.registerNextOperatorSpec(opSpec);
-    otherStreams.forEach(other -> ((MessageStreamImpl<M>) other).getOperatorSpec().registerNextOperatorSpec(opSpec));
-    return new MessageStreamImpl<>(this.graph, opSpec);
+    StreamOperatorSpec<M, M> op = OperatorSpecs.createMergeOperatorSpec(opId);
+    this.operatorSpec.registerNextOperatorSpec(op);
+    otherStreams.forEach(other -> ((MessageStreamImpl<M>) other).getOperatorSpec().registerNextOperatorSpec(op));
+    return new MessageStreamImpl<>(this.graph, op);
   }
 
   @Override
-  public <K, V> MessageStream<KV<K, V>> partitionBy(Function<? super M, ? extends K> keyExtractor,
-      Function<? super M, ? extends V> valueExtractor, KVSerde<K, V> serde, String userDefinedId) {
+  public <K, V> MessageStream<KV<K, V>> partitionBy(MapFunction<? super M, ? extends K> keyExtractor,
+      MapFunction<? super M, ? extends V> valueExtractor, KVSerde<K, V> serde, String userDefinedId) {
     String opId = this.graph.getNextOpId(OpCode.PARTITION_BY, userDefinedId);
     IntermediateMessageStreamImpl<KV<K, V>> intermediateStream = this.graph.getIntermediateStream(opId, serde);
     if (!intermediateStream.isKeyed()) {
       // this can only happen when the default serde partitionBy variant is being used
       throw new SamzaException("partitionBy can not be used with a default serde that is not a KVSerde.");
     }
-    PartitionByOperatorSpec<M, K, V> partitionByOperatorSpec =
-        OperatorSpecs.createPartitionByOperatorSpec(
-            intermediateStream.getOutputStream(), keyExtractor, valueExtractor, opId);
+    PartitionByOperatorSpec<M, K, V> partitionByOperatorSpec = OperatorSpecs.createPartitionByOperatorSpec(
+        intermediateStream.getOutputStream(), keyExtractor, valueExtractor, opId);
     this.operatorSpec.registerNextOperatorSpec(partitionByOperatorSpec);
     return intermediateStream;
   }
 
   @Override
-  public <K, V> MessageStream<KV<K, V>> partitionBy(Function<? super M, ? extends K> keyExtractor,
-      Function<? super M, ? extends V> valueExtractor, String userDefinedId) {
+  public <K, V> MessageStream<KV<K, V>> partitionBy(MapFunction<? super M, ? extends K> keyExtractor,
+      MapFunction<? super M, ? extends V> valueExtractor, String userDefinedId) {
     return partitionBy(keyExtractor, valueExtractor, null, userDefinedId);
   }
 
-  /**
-   * Get the {@link OperatorSpec} associated with this {@link MessageStreamImpl}.
-   * @return the {@link OperatorSpec} associated with this {@link MessageStreamImpl}.
-   */
-  protected OperatorSpec<?, M> getOperatorSpec() {
-    return this.operatorSpec;
-  }
-
   @Override
   public <K, V> void sendTo(Table<KV<K, V>> table) {
-    SendToTableOperatorSpec<K, V> op = OperatorSpecs.createSendToTableOperatorSpec(
-        this.operatorSpec, ((TableImpl) table).getTableSpec(), this.graph.getNextOpId(OpCode.SEND_TO));
+    String opId = this.graph.getNextOpId(OpCode.SEND_TO);
+    SendToTableOperatorSpec<K, V> op =
+        OperatorSpecs.createSendToTableOperatorSpec(((TableImpl) table).getTableSpec(), opId);
     this.operatorSpec.registerNextOperatorSpec(op);
   }
 
@@ -215,4 +206,12 @@ public class MessageStreamImpl<M> implements MessageStream<M> {
     return broadcast(null, userDefinedId);
   }
 
+  /**
+   * Get the {@link OperatorSpec} associated with this {@link MessageStreamImpl}.
+   * @return the {@link OperatorSpec} associated with this {@link MessageStreamImpl}.
+   */
+  protected OperatorSpec<?, M> getOperatorSpec() {
+    return this.operatorSpec;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java b/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java
new file mode 100644
index 0000000..ba51c7c
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/OperatorSpecGraph.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.samza.operators.spec.InputOperatorSpec;
+import org.apache.samza.operators.spec.OperatorSpec;
+import org.apache.samza.operators.spec.OutputStreamImpl;
+import org.apache.samza.serializers.SerializableSerde;
+import org.apache.samza.system.StreamSpec;
+import org.apache.samza.table.TableSpec;
+
+
+/**
+ * Defines the serialized format of {@link StreamGraphSpec}. This class encapsulates all getter methods to get the {@link OperatorSpec}
+ * initialized in the {@link StreamGraphSpec} and constructsthe corresponding serialized instances of {@link OperatorSpec}.
+ * The {@link StreamGraphSpec} and {@link OperatorSpec} instances included in this class are considered as immutable and read-only.
+ * The instance of {@link OperatorSpecGraph} should only be used in runtime to construct {@link org.apache.samza.task.StreamOperatorTask}.
+ */
+public class OperatorSpecGraph implements Serializable {
+  // We use a LHM for deterministic order in initializing and closing operators.
+  private final Map<StreamSpec, InputOperatorSpec> inputOperators;
+  private final Map<StreamSpec, OutputStreamImpl> outputStreams;
+  private final Map<TableSpec, TableImpl> tables;
+  private final Set<OperatorSpec> allOpSpecs;
+  private final boolean hasWindowOrJoins;
+
+  // The following objects are transient since they are recreateable.
+  private transient final SerializableSerde<OperatorSpecGraph> opSpecGraphSerde = new SerializableSerde<>();
+  private transient final byte[] serializedOpSpecGraph;
+
+  OperatorSpecGraph(StreamGraphSpec graphSpec) {
+    this.inputOperators = graphSpec.getInputOperators();
+    this.outputStreams = graphSpec.getOutputStreams();
+    this.tables = graphSpec.getTables();
+    this.allOpSpecs = Collections.unmodifiableSet(this.findAllOperatorSpecs());
+    this.hasWindowOrJoins = checkWindowOrJoins();
+    this.serializedOpSpecGraph = opSpecGraphSerde.toBytes(this);
+  }
+
+  public Map<StreamSpec, InputOperatorSpec> getInputOperators() {
+    return inputOperators;
+  }
+
+  public Map<StreamSpec, OutputStreamImpl> getOutputStreams() {
+    return outputStreams;
+  }
+
+  public Map<TableSpec, TableImpl> getTables() {
+    return tables;
+  }
+
+  /**
+   * Get all {@link OperatorSpec}s available in this {@link StreamGraphSpec}
+   *
+   * @return all available {@link OperatorSpec}s
+   */
+  public Collection<OperatorSpec> getAllOperatorSpecs() {
+    return allOpSpecs;
+  }
+
+  /**
+   * Returns <tt>true</tt> iff this {@link StreamGraphSpec} contains a join or a window operator
+   *
+   * @return  <tt>true</tt> iff this {@link StreamGraphSpec} contains a join or a window operator
+   */
+  public boolean hasWindowOrJoins() {
+    return hasWindowOrJoins;
+  }
+
+  /**
+   * Returns a deserialized {@link OperatorSpecGraph} as a copy from this instance of {@link OperatorSpecGraph}
+   * This is used to create per-task instance of {@link OperatorSpecGraph} when instantiating task instances.
+   *
+   * @return a copy of this {@link OperatorSpecGraph} object via deserialization
+   */
+  public OperatorSpecGraph clone() {
+    if (opSpecGraphSerde == null) {
+      throw new IllegalStateException("Cannot clone from an already deserialized OperatorSpecGraph.");
+    }
+    return opSpecGraphSerde.fromBytes(serializedOpSpecGraph);
+  }
+
+  private HashSet<OperatorSpec> findAllOperatorSpecs() {
+    Collection<InputOperatorSpec> inputOperatorSpecs = this.inputOperators.values();
+    HashSet<OperatorSpec> operatorSpecs = new HashSet<>();
+    for (InputOperatorSpec inputOperatorSpec : inputOperatorSpecs) {
+      operatorSpecs.add(inputOperatorSpec);
+      doGetOperatorSpecs(inputOperatorSpec, operatorSpecs);
+    }
+    return operatorSpecs;
+  }
+
+  private void doGetOperatorSpecs(OperatorSpec operatorSpec, Set<OperatorSpec> specs) {
+    Collection<OperatorSpec> registeredOperatorSpecs = operatorSpec.getRegisteredOperatorSpecs();
+    for (OperatorSpec registeredOperatorSpec : registeredOperatorSpecs) {
+      specs.add(registeredOperatorSpec);
+      doGetOperatorSpecs(registeredOperatorSpec, specs);
+    }
+  }
+
+  private boolean checkWindowOrJoins() {
+    Set<OperatorSpec> windowOrJoinSpecs = allOpSpecs.stream()
+        .filter(spec -> spec.getOpCode() == OperatorSpec.OpCode.WINDOW || spec.getOpCode() == OperatorSpec.OpCode.JOIN)
+        .collect(Collectors.toSet());
+
+    return windowOrJoinSpecs.size() != 0;
+  }
+
+}


[07/10] samza git commit: SAMZA-1659: Serializable OperatorSpec

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 dac4e94..602b595 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
@@ -21,9 +21,8 @@ package org.apache.samza.operators;
 import com.google.common.collect.ImmutableSet;
 import org.apache.samza.Partition;
 import org.apache.samza.SamzaException;
-import org.apache.samza.application.StreamApplication;
 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.metrics.MetricsRegistryMap;
 import org.apache.samza.operators.functions.JoinFunction;
@@ -44,18 +43,24 @@ import org.apache.samza.task.TaskCoordinator;
 import org.apache.samza.testUtils.TestClock;
 import org.apache.samza.util.Clock;
 import org.apache.samza.util.SystemClock;
+import org.junit.Before;
 import org.junit.Test;
 
+import java.io.IOException;
 import java.time.Duration;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.anyString;
+import static org.junit.Assert.assertEquals;
 import static org.mockito.Matchers.eq;
+import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 public class TestJoinOperator {
@@ -64,10 +69,22 @@ public class TestJoinOperator {
   private final TaskCoordinator taskCoordinator = mock(TaskCoordinator.class);
   private final Set<Integer> numbers = ImmutableSet.of(1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
 
+  private Config config;
+
+  @Before
+  public void setUp() {
+    Map<String, String> mapConfig = new HashMap<>();
+    mapConfig.put("app.runner.class", "org.apache.samza.runtime.LocalApplicationRunner");
+    mapConfig.put("job.default.system", "insystem");
+    mapConfig.put("job.name", "jobName");
+    mapConfig.put("job.id", "jobId");
+    config = new MapConfig(mapConfig);
+  }
+
   @Test
   public void join() throws Exception {
-    TestJoinStreamApplication app = new TestJoinStreamApplication(new TestJoinFunction());
-    StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), app);
+    StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction());
+    StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec);
     List<Integer> output = new ArrayList<>();
     MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage());
 
@@ -82,43 +99,42 @@ public class TestJoinOperator {
 
   @Test(expected = SamzaException.class)
   public void joinWithSelfThrowsException() throws Exception {
-    StreamApplication app = new StreamApplication() {
-      @Override
-      public void init(StreamGraph graph, Config config) {
-        IntegerSerde integerSerde = new IntegerSerde();
-        KVSerde<Integer, Integer> kvSerde = KVSerde.of(integerSerde, integerSerde);
-        MessageStream<KV<Integer, Integer>> inStream = graph.getInputStream("instream", kvSerde);
-
-        inStream.join(inStream, new TestJoinFunction(), integerSerde, kvSerde, kvSerde, JOIN_TTL, "join");
-      }
-    };
-
-    createStreamOperatorTask(new SystemClock(), app); // should throw an exception
+    config.put("streams.instream.system", "insystem");
+
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mock(ApplicationRunner.class), config);
+    IntegerSerde integerSerde = new IntegerSerde();
+    KVSerde<Integer, Integer> kvSerde = KVSerde.of(integerSerde, integerSerde);
+    MessageStream<KV<Integer, Integer>> inStream = graphSpec.getInputStream("instream", kvSerde);
+
+    inStream.join(inStream, new TestJoinFunction(), integerSerde, kvSerde, kvSerde, JOIN_TTL, "join");
+
+    createStreamOperatorTask(new SystemClock(), graphSpec); // should throw an exception
   }
 
   @Test
   public void joinFnInitAndClose() throws Exception {
     TestJoinFunction joinFn = new TestJoinFunction();
-    TestJoinStreamApplication app = new TestJoinStreamApplication(joinFn);
-    StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), app);
-    assertEquals(1, joinFn.getNumInitCalls());
+    StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(joinFn);
+    StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec);
+
     MessageCollector messageCollector = mock(MessageCollector.class);
 
     // push messages to first stream
     numbers.forEach(n -> sot.process(new FirstStreamIME(n, n), messageCollector, taskCoordinator));
 
     // close should not be called till now
-    assertEquals(0, joinFn.getNumCloseCalls());
     sot.close();
 
-    // close should be called from sot.close()
-    assertEquals(1, joinFn.getNumCloseCalls());
+    verify(messageCollector, times(0)).send(any(OutgoingMessageEnvelope.class));
+    // Make sure the joinFn has been copied instead of directly referred by the task instance
+    assertEquals(0, joinFn.getNumInitCalls());
+    assertEquals(0, joinFn.getNumCloseCalls());
   }
 
   @Test
   public void joinReverse() throws Exception {
-    TestJoinStreamApplication app = new TestJoinStreamApplication(new TestJoinFunction());
-    StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), app);
+    StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction());
+    StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec);
     List<Integer> output = new ArrayList<>();
     MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage());
 
@@ -133,8 +149,8 @@ public class TestJoinOperator {
 
   @Test
   public void joinNoMatch() throws Exception {
-    TestJoinStreamApplication app = new TestJoinStreamApplication(new TestJoinFunction());
-    StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), app);
+    StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction());
+    StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec);
     List<Integer> output = new ArrayList<>();
     MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage());
 
@@ -148,8 +164,8 @@ public class TestJoinOperator {
 
   @Test
   public void joinNoMatchReverse() throws Exception {
-    TestJoinStreamApplication app = new TestJoinStreamApplication(new TestJoinFunction());
-    StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), app);
+    StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction());
+    StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec);
     List<Integer> output = new ArrayList<>();
     MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage());
 
@@ -163,8 +179,8 @@ public class TestJoinOperator {
 
   @Test
   public void joinRetainsLatestMessageForKey() throws Exception {
-    TestJoinStreamApplication app = new TestJoinStreamApplication(new TestJoinFunction());
-    StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), app);
+    StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction());
+    StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec);
     List<Integer> output = new ArrayList<>();
     MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage());
 
@@ -181,8 +197,8 @@ public class TestJoinOperator {
 
   @Test
   public void joinRetainsLatestMessageForKeyReverse() throws Exception {
-    TestJoinStreamApplication app = new TestJoinStreamApplication(new TestJoinFunction());
-    StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), app);
+    StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction());
+    StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec);
     List<Integer> output = new ArrayList<>();
     MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage());
 
@@ -199,8 +215,8 @@ public class TestJoinOperator {
 
   @Test
   public void joinRetainsMatchedMessages() throws Exception {
-    TestJoinStreamApplication app = new TestJoinStreamApplication(new TestJoinFunction());
-    StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), app);
+    StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction());
+    StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec);
     List<Integer> output = new ArrayList<>();
     MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage());
 
@@ -222,8 +238,8 @@ public class TestJoinOperator {
 
   @Test
   public void joinRetainsMatchedMessagesReverse() throws Exception {
-    TestJoinStreamApplication app = new TestJoinStreamApplication(new TestJoinFunction());
-    StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), app);
+    StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction());
+    StreamOperatorTask sot = createStreamOperatorTask(new SystemClock(), graphSpec);
     List<Integer> output = new ArrayList<>();
     MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage());
 
@@ -246,8 +262,8 @@ public class TestJoinOperator {
   @Test
   public void joinRemovesExpiredMessages() throws Exception {
     TestClock testClock = new TestClock();
-    TestJoinStreamApplication app = new TestJoinStreamApplication(new TestJoinFunction());
-    StreamOperatorTask sot = createStreamOperatorTask(testClock, app);
+    StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction());
+    StreamOperatorTask sot = createStreamOperatorTask(testClock, graphSpec);
     List<Integer> output = new ArrayList<>();
     MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage());
 
@@ -266,8 +282,8 @@ public class TestJoinOperator {
   @Test
   public void joinRemovesExpiredMessagesReverse() throws Exception {
     TestClock testClock = new TestClock();
-    TestJoinStreamApplication app = new TestJoinStreamApplication(new TestJoinFunction());
-    StreamOperatorTask sot = createStreamOperatorTask(testClock, app);
+    StreamGraphSpec graphSpec = this.getTestJoinStreamGraph(new TestJoinFunction());
+    StreamOperatorTask sot = createStreamOperatorTask(testClock, graphSpec);
     List<Integer> output = new ArrayList<>();
     MessageCollector messageCollector = envelope -> output.add((Integer) envelope.getMessage());
 
@@ -283,15 +299,12 @@ public class TestJoinOperator {
     assertTrue(output.isEmpty());
   }
 
-  private StreamOperatorTask createStreamOperatorTask(Clock clock, StreamApplication app) throws Exception {
-    ApplicationRunner runner = mock(ApplicationRunner.class);
-    when(runner.getStreamSpec("instream")).thenReturn(new StreamSpec("instream", "instream", "insystem"));
-    when(runner.getStreamSpec("instream2")).thenReturn(new StreamSpec("instream2", "instream2", "insystem2"));
+  private StreamOperatorTask createStreamOperatorTask(Clock clock, StreamGraphSpec graphSpec) throws Exception {
 
     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))));
+            new SystemStreamPartition("insystem", "instream2", new Partition(0))));
     when(taskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap());
     // need to return different stores for left and right side
     IntegerSerde integerSerde = new IntegerSerde();
@@ -301,35 +314,30 @@ public class TestJoinOperator {
     when(taskContext.getStore(eq("jobName-jobId-join-j1-R")))
         .thenReturn(new TestInMemoryStore(integerSerde, timestampedValueSerde));
 
-    Config config = mock(Config.class);
-    when(config.get(JobConfig.JOB_NAME())).thenReturn("jobName");
-    when(config.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("jobId");
-
-    StreamOperatorTask sot = new StreamOperatorTask(app, runner, clock);
+    StreamOperatorTask sot = new StreamOperatorTask(graphSpec.getOperatorSpecGraph(), graphSpec.getContextManager(), clock);
     sot.init(config, taskContext);
     return sot;
   }
 
-  private static class TestJoinStreamApplication implements StreamApplication {
-
-    private final TestJoinFunction joinFn;
-
-    TestJoinStreamApplication(TestJoinFunction joinFn) {
-      this.joinFn = joinFn;
-    }
+  private StreamGraphSpec getTestJoinStreamGraph(TestJoinFunction joinFn) throws IOException {
+    ApplicationRunner runner = mock(ApplicationRunner.class);
+    when(runner.getStreamSpec("instream")).thenReturn(new StreamSpec("instream", "instream", "insystem"));
+    when(runner.getStreamSpec("instream2")).thenReturn(new StreamSpec("instream2", "instream2", "insystem"));
 
-    @Override
-    public void init(StreamGraph graph, Config config) {
-      IntegerSerde integerSerde = new IntegerSerde();
-      KVSerde<Integer, Integer> kvSerde = KVSerde.of(integerSerde, integerSerde);
-      MessageStream<KV<Integer, Integer>> inStream = graph.getInputStream("instream", kvSerde);
-      MessageStream<KV<Integer, Integer>> inStream2 = graph.getInputStream("instream2", kvSerde);
-
-      SystemStream outputSystemStream = new SystemStream("outputSystem", "outputStream");
-      inStream
-          .join(inStream2, joinFn, integerSerde, kvSerde, kvSerde, JOIN_TTL, "j1")
-          .sink((m, mc, tc) -> mc.send(new OutgoingMessageEnvelope(outputSystemStream, m)));
-    }
+    StreamGraphSpec graphSpec = new StreamGraphSpec(runner, config);
+    IntegerSerde integerSerde = new IntegerSerde();
+    KVSerde<Integer, Integer> kvSerde = KVSerde.of(integerSerde, integerSerde);
+    MessageStream<KV<Integer, Integer>> inStream = graphSpec.getInputStream("instream", kvSerde);
+    MessageStream<KV<Integer, Integer>> inStream2 = graphSpec.getInputStream("instream2", kvSerde);
+
+    inStream
+        .join(inStream2, joinFn, integerSerde, kvSerde, kvSerde, JOIN_TTL, "j1")
+        .sink((message, messageCollector, taskCoordinator) -> {
+            SystemStream outputSystemStream = new SystemStream("outputSystem", "outputStream");
+            messageCollector.send(new OutgoingMessageEnvelope(outputSystemStream, message));
+          });
+
+    return graphSpec;
   }
 
   private static class TestJoinFunction
@@ -380,7 +388,7 @@ public class TestJoinOperator {
 
   private static class SecondStreamIME extends IncomingMessageEnvelope {
     SecondStreamIME(Integer key, Integer value) {
-      super(new SystemStreamPartition("insystem2", "instream2", new Partition(0)), "1", key, value);
+      super(new SystemStreamPartition("insystem", "instream2", new Partition(0)), "1", key, value);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java b/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
index 96e234e..fff85e8 100644
--- a/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
+++ b/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
@@ -18,11 +18,11 @@
  */
 package org.apache.samza.operators;
 
+import com.google.common.collect.ImmutableList;
+import java.io.IOException;
 import java.time.Duration;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.function.Function;
-import java.util.function.Supplier;
 
 import org.apache.samza.operators.data.TestMessageEnvelope;
 import org.apache.samza.operators.data.TestOutputMessageEnvelope;
@@ -32,6 +32,7 @@ import org.apache.samza.operators.functions.FoldLeftFunction;
 import org.apache.samza.operators.functions.JoinFunction;
 import org.apache.samza.operators.functions.MapFunction;
 import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.operators.functions.SupplierFunction;
 import org.apache.samza.operators.functions.StreamTableJoinFunction;
 import org.apache.samza.operators.spec.JoinOperatorSpec;
 import org.apache.samza.operators.spec.OperatorSpec;
@@ -54,8 +55,6 @@ import org.apache.samza.table.TableSpec;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 
-import com.google.common.collect.ImmutableList;
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -71,7 +70,7 @@ public class TestMessageStreamImpl {
 
   @Test
   public void testMap() {
-    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    StreamGraphSpec mockGraph = mock(StreamGraphSpec.class);
     OperatorSpec mockOpSpec = mock(OperatorSpec.class);
     MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec);
 
@@ -96,7 +95,7 @@ public class TestMessageStreamImpl {
 
   @Test
   public void testFlatMap() {
-    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    StreamGraphSpec mockGraph = mock(StreamGraphSpec.class);
     OperatorSpec mockOpSpec = mock(OperatorSpec.class);
     MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec);
 
@@ -113,7 +112,7 @@ public class TestMessageStreamImpl {
 
   @Test
   public void testFlatMapWithRelaxedTypes() {
-    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    StreamGraphSpec mockGraph = mock(StreamGraphSpec.class);
     OperatorSpec mockOpSpec = mock(OperatorSpec.class);
     MessageStreamImpl<TestInputMessageEnvelope> inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec);
 
@@ -133,7 +132,7 @@ public class TestMessageStreamImpl {
 
   @Test
   public void testFilter() {
-    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    StreamGraphSpec mockGraph = mock(StreamGraphSpec.class);
     OperatorSpec mockOpSpec = mock(OperatorSpec.class);
     MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec);
 
@@ -158,7 +157,7 @@ public class TestMessageStreamImpl {
 
   @Test
   public void testSink() {
-    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    StreamGraphSpec mockGraph = mock(StreamGraphSpec.class);
     OperatorSpec mockOpSpec = mock(OperatorSpec.class);
     MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec);
 
@@ -175,7 +174,7 @@ public class TestMessageStreamImpl {
 
   @Test
   public void testSendTo() {
-    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    StreamGraphSpec mockGraph = mock(StreamGraphSpec.class);
     OperatorSpec mockOpSpec = mock(OperatorSpec.class);
     MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec);
     OutputStreamImpl<TestMessageEnvelope> mockOutputStreamImpl = mock(OutputStreamImpl.class);
@@ -200,8 +199,8 @@ public class TestMessageStreamImpl {
   }
 
   @Test
-  public void testRepartition() {
-    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+  public void testPartitionBy() throws IOException {
+    StreamGraphSpec mockGraph = mock(StreamGraphSpec.class);
     OperatorSpec mockOpSpec = mock(OperatorSpec.class);
     String mockOpName = "mockName";
     when(mockGraph.getNextOpId(anyObject(), anyObject())).thenReturn(mockOpName);
@@ -215,8 +214,8 @@ public class TestMessageStreamImpl {
     when(mockIntermediateStream.isKeyed()).thenReturn(true);
 
     MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec);
-    Function mockKeyFunction = mock(Function.class);
-    Function mockValueFunction = mock(Function.class);
+    MapFunction mockKeyFunction = mock(MapFunction.class);
+    MapFunction mockValueFunction = mock(MapFunction.class);
     inputStream.partitionBy(mockKeyFunction, mockValueFunction, mockKVSerde, "p1");
 
     ArgumentCaptor<OperatorSpec> registeredOpCaptor = ArgumentCaptor.forClass(OperatorSpec.class);
@@ -232,7 +231,7 @@ public class TestMessageStreamImpl {
 
   @Test
   public void testRepartitionWithoutSerde() {
-    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    StreamGraphSpec mockGraph = mock(StreamGraphSpec.class);
     OperatorSpec mockOpSpec = mock(OperatorSpec.class);
     String mockOpName = "mockName";
     when(mockGraph.getNextOpId(anyObject(), anyObject())).thenReturn(mockOpName);
@@ -245,8 +244,8 @@ public class TestMessageStreamImpl {
     when(mockIntermediateStream.isKeyed()).thenReturn(true);
 
     MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec);
-    Function mockKeyFunction = mock(Function.class);
-    Function mockValueFunction = mock(Function.class);
+    MapFunction mockKeyFunction = mock(MapFunction.class);
+    MapFunction mockValueFunction = mock(MapFunction.class);
     inputStream.partitionBy(mockKeyFunction, mockValueFunction, "p1");
 
     ArgumentCaptor<OperatorSpec> registeredOpCaptor = ArgumentCaptor.forClass(OperatorSpec.class);
@@ -262,18 +261,17 @@ public class TestMessageStreamImpl {
 
   @Test
   public void testWindowWithRelaxedTypes() throws Exception {
-    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    StreamGraphSpec mockGraph = mock(StreamGraphSpec.class);
     OperatorSpec mockOpSpec = mock(OperatorSpec.class);
     MessageStream<TestInputMessageEnvelope> inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec);
 
-    Function<TestMessageEnvelope, String> keyExtractor = m -> m.getKey();
+    MapFunction<TestMessageEnvelope, String> keyExtractor = m -> m.getKey();
     FoldLeftFunction<TestMessageEnvelope, Integer> aggregator = (m, c) -> c + 1;
-    Supplier<Integer> initialValue = () -> 0;
+    SupplierFunction<Integer> initialValue = () -> 0;
 
     // should compile since TestMessageEnvelope (input for functions) is base class of TestInputMessageEnvelope (M)
-    Window<TestInputMessageEnvelope, String, Integer> window =
-        Windows.keyedTumblingWindow(keyExtractor, Duration.ofHours(1), initialValue, aggregator,
-            null, mock(Serde.class));
+    Window<TestInputMessageEnvelope, String, Integer> window = Windows
+        .keyedTumblingWindow(keyExtractor, Duration.ofHours(1), initialValue, aggregator, null, mock(Serde.class));
     MessageStream<WindowPane<String, Integer>> windowedStream = inputStream.window(window, "w1");
 
     ArgumentCaptor<OperatorSpec> registeredOpCaptor = ArgumentCaptor.forClass(OperatorSpec.class);
@@ -287,7 +285,7 @@ public class TestMessageStreamImpl {
 
   @Test
   public void testJoin() {
-    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    StreamGraphSpec mockGraph = mock(StreamGraphSpec.class);
     OperatorSpec leftInputOpSpec = mock(OperatorSpec.class);
     MessageStreamImpl<TestMessageEnvelope> source1 = new MessageStreamImpl<>(mockGraph, leftInputOpSpec);
     OperatorSpec rightInputOpSpec = mock(OperatorSpec.class);
@@ -319,7 +317,7 @@ public class TestMessageStreamImpl {
 
   @Test
   public void testSendToTable() {
-    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    StreamGraphSpec mockGraph = mock(StreamGraphSpec.class);
     OperatorSpec inputOpSpec = mock(OperatorSpec.class);
     MessageStreamImpl<TestMessageEnvelope> source = new MessageStreamImpl<>(mockGraph, inputOpSpec);
 
@@ -336,13 +334,12 @@ public class TestMessageStreamImpl {
     SendToTableOperatorSpec sendToTableOperatorSpec = (SendToTableOperatorSpec) registeredOpSpec;
 
     assertEquals(OpCode.SEND_TO, sendToTableOperatorSpec.getOpCode());
-    assertEquals(inputOpSpec, sendToTableOperatorSpec.getInputOpSpec());
     assertEquals(tableSpec, sendToTableOperatorSpec.getTableSpec());
   }
 
   @Test
   public void testStreamTableJoin() {
-    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    StreamGraphSpec mockGraph = mock(StreamGraphSpec.class);
     OperatorSpec leftInputOpSpec = mock(OperatorSpec.class);
     MessageStreamImpl<KV<String, TestMessageEnvelope>> source1 = new MessageStreamImpl<>(mockGraph, leftInputOpSpec);
     OperatorSpec rightInputOpSpec = mock(OperatorSpec.class);
@@ -370,7 +367,7 @@ public class TestMessageStreamImpl {
 
   @Test
   public void testMerge() {
-    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    StreamGraphSpec mockGraph = mock(StreamGraphSpec.class);
     OperatorSpec mockOpSpec1 = mock(OperatorSpec.class);
     MessageStream<TestMessageEnvelope> inputStream = new MessageStreamImpl<>(mockGraph, mockOpSpec1);
 
@@ -410,7 +407,7 @@ public class TestMessageStreamImpl {
 
   @Test
   public void testMergeWithRelaxedTypes() {
-    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    StreamGraphSpec mockGraph = mock(StreamGraphSpec.class);
     MessageStream<TestMessageEnvelope> inputStream = new MessageStreamImpl<>(mockGraph, mock(OperatorSpec.class));
 
     // other streams have the same message type T as input stream message type M

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java b/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java
new file mode 100644
index 0000000..2be88ca
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/operators/TestOperatorSpecGraph.java
@@ -0,0 +1,185 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for THE
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators;
+
+import java.io.IOException;
+import java.io.NotSerializableException;
+import java.io.ObjectInputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import org.apache.samza.SamzaException;
+import org.apache.samza.operators.functions.TimerFunction;
+import org.apache.samza.operators.functions.WatermarkFunction;
+import org.apache.samza.operators.spec.InputOperatorSpec;
+import org.apache.samza.operators.spec.OperatorSpec;
+import org.apache.samza.operators.spec.OperatorSpecTestUtils;
+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.operators.spec.SinkOperatorSpec;
+import org.apache.samza.operators.spec.StreamOperatorSpec;
+import org.apache.samza.serializers.NoOpSerde;
+import org.apache.samza.system.StreamSpec;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+
+/**
+ * Unit tests for {@link OperatorSpecGraph}
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest(OperatorSpec.class)
+public class TestOperatorSpecGraph {
+
+  private StreamGraphSpec mockGraph;
+  private Map<StreamSpec, InputOperatorSpec> inputOpSpecMap;
+  private Map<StreamSpec, OutputStreamImpl> outputStrmMap;
+  private Set<OperatorSpec> allOpSpecs;
+
+  @Before
+  public void setUp() {
+    this.mockGraph = mock(StreamGraphSpec.class);
+
+    /**
+     * Setup two linear transformation pipelines:
+     * 1) input1 --> filter --> sendTo
+     * 2) input2 --> map --> sink
+     */
+    StreamSpec testInputSpec = new StreamSpec("test-input-1", "test-input-1", "kafka");
+    InputOperatorSpec testInput = new InputOperatorSpec(testInputSpec, new NoOpSerde(), new NoOpSerde(), true, "test-input-1");
+    StreamOperatorSpec filterOp = OperatorSpecs.createFilterOperatorSpec(m -> true, "test-filter-2");
+    StreamSpec testOutputSpec = new StreamSpec("test-output-1", "test-output-1", "kafka");
+    OutputStreamImpl outputStream1 = new OutputStreamImpl(testOutputSpec, null, null, true);
+    OutputOperatorSpec outputSpec = OperatorSpecs.createSendToOperatorSpec(outputStream1, "test-output-3");
+    testInput.registerNextOperatorSpec(filterOp);
+    filterOp.registerNextOperatorSpec(outputSpec);
+    StreamSpec testInputSpec2 = new StreamSpec("test-input-2", "test-input-2", "kafka");
+    InputOperatorSpec testInput2 = new InputOperatorSpec(testInputSpec2, new NoOpSerde(), new NoOpSerde(), true, "test-input-4");
+    StreamOperatorSpec testMap = OperatorSpecs.createMapOperatorSpec(m -> m, "test-map-5");
+    SinkOperatorSpec testSink = OperatorSpecs.createSinkOperatorSpec((m, mc, tc) -> { }, "test-sink-6");
+    testInput2.registerNextOperatorSpec(testMap);
+    testMap.registerNextOperatorSpec(testSink);
+
+    this.inputOpSpecMap = new LinkedHashMap<>();
+    inputOpSpecMap.put(testInputSpec, testInput);
+    inputOpSpecMap.put(testInputSpec2, testInput2);
+    this.outputStrmMap = new LinkedHashMap<>();
+    outputStrmMap.put(testOutputSpec, outputStream1);
+    when(mockGraph.getInputOperators()).thenReturn(Collections.unmodifiableMap(inputOpSpecMap));
+    when(mockGraph.getOutputStreams()).thenReturn(Collections.unmodifiableMap(outputStrmMap));
+    this.allOpSpecs = new HashSet<OperatorSpec>() { {
+        this.add(testInput);
+        this.add(filterOp);
+        this.add(outputSpec);
+        this.add(testInput2);
+        this.add(testMap);
+        this.add(testSink);
+      } };
+  }
+
+  @After
+  public void tearDown() {
+    this.mockGraph = null;
+    this.inputOpSpecMap = null;
+    this.outputStrmMap = null;
+    this.allOpSpecs = null;
+  }
+
+  @Test
+  public void testConstructor() {
+    OperatorSpecGraph specGraph = new OperatorSpecGraph(mockGraph);
+    assertEquals(specGraph.getInputOperators(), inputOpSpecMap);
+    assertEquals(specGraph.getOutputStreams(), outputStrmMap);
+    assertTrue(specGraph.getTables().isEmpty());
+    assertTrue(!specGraph.hasWindowOrJoins());
+    assertEquals(specGraph.getAllOperatorSpecs(), this.allOpSpecs);
+  }
+
+  @Test
+  public void testClone() {
+    OperatorSpecGraph operatorSpecGraph = new OperatorSpecGraph(mockGraph);
+    OperatorSpecGraph clonedSpecGraph = operatorSpecGraph.clone();
+    OperatorSpecTestUtils.assertClonedGraph(operatorSpecGraph, clonedSpecGraph);
+  }
+
+  @Test(expected = NotSerializableException.class)
+  public void testCloneWithSerializationError() throws Throwable {
+    OperatorSpec mockFailedOpSpec = PowerMockito.mock(OperatorSpec.class);
+    when(mockFailedOpSpec.getOpId()).thenReturn("test-failed-op-4");
+    allOpSpecs.add(mockFailedOpSpec);
+    inputOpSpecMap.values().stream().findFirst().get().registerNextOperatorSpec(mockFailedOpSpec);
+
+    //failed with serialization error
+    try {
+      new OperatorSpecGraph(mockGraph);
+      fail("Should have failed with serialization error");
+    } catch (SamzaException se) {
+      throw se.getCause();
+    }
+  }
+
+  @Test(expected = IOException.class)
+  public void testCloneWithDeserializationError() throws Throwable {
+    TestDeserializeOperatorSpec testOp = new TestDeserializeOperatorSpec(OperatorSpec.OpCode.MAP, "test-failed-op-4");
+    this.allOpSpecs.add(testOp);
+    inputOpSpecMap.values().stream().findFirst().get().registerNextOperatorSpec(testOp);
+
+    OperatorSpecGraph operatorSpecGraph = new OperatorSpecGraph(mockGraph);
+    //failed with serialization error
+    try {
+      operatorSpecGraph.clone();
+      fail("Should have failed with serialization error");
+    } catch (SamzaException se) {
+      throw se.getCause();
+    }
+  }
+
+  private static class TestDeserializeOperatorSpec extends OperatorSpec {
+
+    public TestDeserializeOperatorSpec(OpCode opCode, String opId) {
+      super(opCode, opId);
+    }
+
+    private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException {
+      throw new IOException("Raise IOException to cause deserialization failure");
+    }
+
+    @Override
+    public WatermarkFunction getWatermarkFn() {
+      return null;
+    }
+
+    @Override
+    public TimerFunction getTimerFn() {
+      return null;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphImpl.java b/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphImpl.java
deleted file mode 100644
index 3bb44b5..0000000
--- a/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphImpl.java
+++ /dev/null
@@ -1,601 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for THE
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-
-import org.apache.samza.SamzaException;
-import org.apache.samza.config.Config;
-import org.apache.samza.config.JobConfig;
-import org.apache.samza.operators.data.TestMessageEnvelope;
-import org.apache.samza.operators.spec.InputOperatorSpec;
-import org.apache.samza.operators.spec.OperatorSpec.OpCode;
-import org.apache.samza.operators.spec.OutputStreamImpl;
-import org.apache.samza.operators.stream.IntermediateMessageStreamImpl;
-import org.apache.samza.runtime.ApplicationRunner;
-import org.apache.samza.serializers.KVSerde;
-import org.apache.samza.serializers.NoOpSerde;
-import org.apache.samza.serializers.Serde;
-import org.apache.samza.system.StreamSpec;
-import org.apache.samza.table.TableSpec;
-import org.junit.Test;
-
-import com.google.common.collect.ImmutableList;
-import junit.framework.Assert;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-public class TestStreamGraphImpl {
-
-  @Test
-  public void testGetInputStreamWithValueSerde() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    StreamSpec mockStreamSpec = mock(StreamSpec.class);
-    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mock(Config.class));
-
-    Serde mockValueSerde = mock(Serde.class);
-    MessageStream<TestMessageEnvelope> inputStream = graph.getInputStream("test-stream-1", mockValueSerde);
-
-    InputOperatorSpec<String, TestMessageEnvelope> inputOpSpec =
-        (InputOperatorSpec) ((MessageStreamImpl<TestMessageEnvelope>) inputStream).getOperatorSpec();
-    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
-    assertEquals(graph.getInputOperators().get(mockStreamSpec), inputOpSpec);
-    assertEquals(mockStreamSpec, inputOpSpec.getStreamSpec());
-    assertTrue(inputOpSpec.getKeySerde() instanceof NoOpSerde);
-    assertEquals(mockValueSerde, inputOpSpec.getValueSerde());
-  }
-
-  @Test
-  public void testGetInputStreamWithKeyValueSerde() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    StreamSpec mockStreamSpec = mock(StreamSpec.class);
-    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mock(Config.class));
-
-    KVSerde mockKVSerde = mock(KVSerde.class);
-    Serde mockKeySerde = mock(Serde.class);
-    Serde mockValueSerde = mock(Serde.class);
-    doReturn(mockKeySerde).when(mockKVSerde).getKeySerde();
-    doReturn(mockValueSerde).when(mockKVSerde).getValueSerde();
-    MessageStream<TestMessageEnvelope> inputStream = graph.getInputStream("test-stream-1", mockKVSerde);
-
-    InputOperatorSpec<String, TestMessageEnvelope> inputOpSpec =
-        (InputOperatorSpec) ((MessageStreamImpl<TestMessageEnvelope>) inputStream).getOperatorSpec();
-    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
-    assertEquals(graph.getInputOperators().get(mockStreamSpec), inputOpSpec);
-    assertEquals(mockStreamSpec, inputOpSpec.getStreamSpec());
-    assertEquals(mockKeySerde, inputOpSpec.getKeySerde());
-    assertEquals(mockValueSerde, inputOpSpec.getValueSerde());
-  }
-
-  @Test(expected = NullPointerException.class)
-  public void testGetInputStreamWithNullSerde() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    StreamSpec mockStreamSpec = mock(StreamSpec.class);
-    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mock(Config.class));
-
-    graph.getInputStream("test-stream-1", null);
-  }
-
-  @Test
-  public void testGetInputStreamWithDefaultValueSerde() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    StreamSpec mockStreamSpec = mock(StreamSpec.class);
-    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mock(Config.class));
-
-    Serde mockValueSerde = mock(Serde.class);
-    graph.setDefaultSerde(mockValueSerde);
-    MessageStream<TestMessageEnvelope> inputStream = graph.getInputStream("test-stream-1");
-
-    InputOperatorSpec<String, TestMessageEnvelope> inputOpSpec =
-        (InputOperatorSpec) ((MessageStreamImpl<TestMessageEnvelope>) inputStream).getOperatorSpec();
-    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
-    assertEquals(graph.getInputOperators().get(mockStreamSpec), inputOpSpec);
-    assertEquals(mockStreamSpec, inputOpSpec.getStreamSpec());
-    assertTrue(inputOpSpec.getKeySerde() instanceof NoOpSerde);
-    assertEquals(mockValueSerde, inputOpSpec.getValueSerde());
-  }
-
-  @Test
-  public void testGetInputStreamWithDefaultKeyValueSerde() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    StreamSpec mockStreamSpec = mock(StreamSpec.class);
-    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mock(Config.class));
-
-    KVSerde mockKVSerde = mock(KVSerde.class);
-    Serde mockKeySerde = mock(Serde.class);
-    Serde mockValueSerde = mock(Serde.class);
-    doReturn(mockKeySerde).when(mockKVSerde).getKeySerde();
-    doReturn(mockValueSerde).when(mockKVSerde).getValueSerde();
-    graph.setDefaultSerde(mockKVSerde);
-    MessageStream<TestMessageEnvelope> inputStream = graph.getInputStream("test-stream-1");
-
-    InputOperatorSpec<String, TestMessageEnvelope> inputOpSpec =
-        (InputOperatorSpec) ((MessageStreamImpl<TestMessageEnvelope>) inputStream).getOperatorSpec();
-    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
-    assertEquals(graph.getInputOperators().get(mockStreamSpec), inputOpSpec);
-    assertEquals(mockStreamSpec, inputOpSpec.getStreamSpec());
-    assertEquals(mockKeySerde, inputOpSpec.getKeySerde());
-    assertEquals(mockValueSerde, inputOpSpec.getValueSerde());
-  }
-
-  @Test
-  public void testGetInputStreamWithDefaultDefaultSerde() {
-    // default default serde == user hasn't provided a default serde
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    StreamSpec mockStreamSpec = mock(StreamSpec.class);
-    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mock(Config.class));
-
-    MessageStream<TestMessageEnvelope> inputStream = graph.getInputStream("test-stream-1");
-
-    InputOperatorSpec<String, TestMessageEnvelope> inputOpSpec =
-        (InputOperatorSpec) ((MessageStreamImpl<TestMessageEnvelope>) inputStream).getOperatorSpec();
-    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
-    assertEquals(graph.getInputOperators().get(mockStreamSpec), inputOpSpec);
-    assertEquals(mockStreamSpec, inputOpSpec.getStreamSpec());
-    assertTrue(inputOpSpec.getKeySerde() instanceof NoOpSerde);
-    assertTrue(inputOpSpec.getValueSerde() instanceof NoOpSerde);
-  }
-
-  @Test
-  public void testGetInputStreamWithRelaxedTypes() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    StreamSpec mockStreamSpec = mock(StreamSpec.class);
-    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mock(Config.class));
-
-    MessageStream<TestMessageEnvelope> inputStream = graph.getInputStream("test-stream-1");
-
-    InputOperatorSpec<String, TestMessageEnvelope> inputOpSpec =
-        (InputOperatorSpec) ((MessageStreamImpl<TestMessageEnvelope>) inputStream).getOperatorSpec();
-    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
-    assertEquals(graph.getInputOperators().get(mockStreamSpec), inputOpSpec);
-    assertEquals(mockStreamSpec, inputOpSpec.getStreamSpec());
-  }
-
-  @Test
-  public void testMultipleGetInputStreams() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    StreamSpec mockStreamSpec1 = mock(StreamSpec.class);
-    StreamSpec mockStreamSpec2 = mock(StreamSpec.class);
-    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec1);
-    when(mockRunner.getStreamSpec("test-stream-2")).thenReturn(mockStreamSpec2);
-
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mock(Config.class));
-    MessageStream<Object> inputStream1 = graph.getInputStream("test-stream-1");
-    MessageStream<Object> inputStream2 = graph.getInputStream("test-stream-2");
-
-    InputOperatorSpec<String, TestMessageEnvelope> inputOpSpec1 =
-        (InputOperatorSpec) ((MessageStreamImpl<Object>) inputStream1).getOperatorSpec();
-    InputOperatorSpec<String, TestMessageEnvelope> inputOpSpec2 =
-        (InputOperatorSpec) ((MessageStreamImpl<Object>) inputStream2).getOperatorSpec();
-
-    assertEquals(graph.getInputOperators().size(), 2);
-    assertEquals(graph.getInputOperators().get(mockStreamSpec1), inputOpSpec1);
-    assertEquals(graph.getInputOperators().get(mockStreamSpec2), inputOpSpec2);
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testGetSameInputStreamTwice() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mock(StreamSpec.class));
-
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mock(Config.class));
-    graph.getInputStream("test-stream-1");
-    // should throw exception
-    graph.getInputStream("test-stream-1");
-  }
-
-  @Test
-  public void testGetOutputStreamWithValueSerde() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    StreamSpec mockStreamSpec = mock(StreamSpec.class);
-    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
-
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mock(Config.class));
-
-    Serde mockValueSerde = mock(Serde.class);
-    OutputStream<TestMessageEnvelope> outputStream =
-        graph.getOutputStream("test-stream-1", mockValueSerde);
-
-    OutputStreamImpl<TestMessageEnvelope> outputStreamImpl = (OutputStreamImpl) outputStream;
-    assertEquals(graph.getOutputStreams().get(mockStreamSpec), outputStreamImpl);
-    assertEquals(mockStreamSpec, outputStreamImpl.getStreamSpec());
-    assertTrue(outputStreamImpl.getKeySerde() instanceof NoOpSerde);
-    assertEquals(mockValueSerde, outputStreamImpl.getValueSerde());
-  }
-
-  @Test
-  public void testGetOutputStreamWithKeyValueSerde() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    StreamSpec mockStreamSpec = mock(StreamSpec.class);
-    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
-
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mock(Config.class));
-    KVSerde mockKVSerde = mock(KVSerde.class);
-    Serde mockKeySerde = mock(Serde.class);
-    Serde mockValueSerde = mock(Serde.class);
-    doReturn(mockKeySerde).when(mockKVSerde).getKeySerde();
-    doReturn(mockValueSerde).when(mockKVSerde).getValueSerde();
-    graph.setDefaultSerde(mockKVSerde);
-    OutputStream<TestMessageEnvelope> outputStream = graph.getOutputStream("test-stream-1", mockKVSerde);
-
-    OutputStreamImpl<TestMessageEnvelope> outputStreamImpl = (OutputStreamImpl) outputStream;
-    assertEquals(graph.getOutputStreams().get(mockStreamSpec), outputStreamImpl);
-    assertEquals(mockStreamSpec, outputStreamImpl.getStreamSpec());
-    assertEquals(mockKeySerde, outputStreamImpl.getKeySerde());
-    assertEquals(mockValueSerde, outputStreamImpl.getValueSerde());
-  }
-
-  @Test(expected = NullPointerException.class)
-  public void testGetOutputStreamWithNullSerde() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    StreamSpec mockStreamSpec = mock(StreamSpec.class);
-    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
-
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mock(Config.class));
-
-    graph.getOutputStream("test-stream-1", null);
-  }
-
-  @Test
-  public void testGetOutputStreamWithDefaultValueSerde() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    StreamSpec mockStreamSpec = mock(StreamSpec.class);
-    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
-
-    Serde mockValueSerde = mock(Serde.class);
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mock(Config.class));
-    graph.setDefaultSerde(mockValueSerde);
-    OutputStream<TestMessageEnvelope> outputStream = graph.getOutputStream("test-stream-1");
-
-    OutputStreamImpl<TestMessageEnvelope> outputStreamImpl = (OutputStreamImpl) outputStream;
-    assertEquals(graph.getOutputStreams().get(mockStreamSpec), outputStreamImpl);
-    assertEquals(mockStreamSpec, outputStreamImpl.getStreamSpec());
-    assertTrue(outputStreamImpl.getKeySerde() instanceof NoOpSerde);
-    assertEquals(mockValueSerde, outputStreamImpl.getValueSerde());
-  }
-
-  @Test
-  public void testGetOutputStreamWithDefaultKeyValueSerde() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    StreamSpec mockStreamSpec = mock(StreamSpec.class);
-    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
-
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mock(Config.class));
-    KVSerde mockKVSerde = mock(KVSerde.class);
-    Serde mockKeySerde = mock(Serde.class);
-    Serde mockValueSerde = mock(Serde.class);
-    doReturn(mockKeySerde).when(mockKVSerde).getKeySerde();
-    doReturn(mockValueSerde).when(mockKVSerde).getValueSerde();
-    graph.setDefaultSerde(mockKVSerde);
-
-    OutputStream<TestMessageEnvelope> outputStream = graph.getOutputStream("test-stream-1");
-
-    OutputStreamImpl<TestMessageEnvelope> outputStreamImpl = (OutputStreamImpl) outputStream;
-    assertEquals(graph.getOutputStreams().get(mockStreamSpec), outputStreamImpl);
-    assertEquals(mockStreamSpec, outputStreamImpl.getStreamSpec());
-    assertEquals(mockKeySerde, outputStreamImpl.getKeySerde());
-    assertEquals(mockValueSerde, outputStreamImpl.getValueSerde());
-  }
-
-  @Test
-  public void testGetOutputStreamWithDefaultDefaultSerde() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    StreamSpec mockStreamSpec = mock(StreamSpec.class);
-    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
-
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mock(Config.class));
-
-    OutputStream<TestMessageEnvelope> outputStream = graph.getOutputStream("test-stream-1");
-
-    OutputStreamImpl<TestMessageEnvelope> outputStreamImpl = (OutputStreamImpl) outputStream;
-    assertEquals(graph.getOutputStreams().get(mockStreamSpec), outputStreamImpl);
-    assertEquals(mockStreamSpec, outputStreamImpl.getStreamSpec());
-    assertTrue(outputStreamImpl.getKeySerde() instanceof NoOpSerde);
-    assertTrue(outputStreamImpl.getValueSerde() instanceof NoOpSerde);
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testSetDefaultSerdeAfterGettingStreams() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mock(StreamSpec.class));
-
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mock(Config.class));
-    graph.getInputStream("test-stream-1");
-    graph.setDefaultSerde(mock(Serde.class)); // should throw exception
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testSetDefaultSerdeAfterGettingOutputStream() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mock(StreamSpec.class));
-
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mock(Config.class));
-    graph.getOutputStream("test-stream-1");
-    graph.setDefaultSerde(mock(Serde.class)); // should throw exception
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testSetDefaultSerdeAfterGettingIntermediateStream() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mock(StreamSpec.class));
-
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mock(Config.class));
-    graph.getIntermediateStream("test-stream-1", null);
-    graph.setDefaultSerde(mock(Serde.class)); // should throw exception
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testGetSameOutputStreamTwice() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mock(StreamSpec.class));
-
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mock(Config.class));
-    graph.getOutputStream("test-stream-1");
-    graph.getOutputStream("test-stream-1"); // should throw exception
-  }
-
-  @Test
-  public void testGetIntermediateStreamWithValueSerde() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    Config mockConfig = mock(Config.class);
-    StreamSpec mockStreamSpec = mock(StreamSpec.class);
-    String mockStreamName = "mockStreamName";
-    when(mockRunner.getStreamSpec(mockStreamName)).thenReturn(mockStreamSpec);
-
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mockConfig);
-
-    Serde mockValueSerde = mock(Serde.class);
-    IntermediateMessageStreamImpl<TestMessageEnvelope> intermediateStreamImpl =
-        graph.getIntermediateStream(mockStreamName, mockValueSerde);
-
-    assertEquals(graph.getInputOperators().get(mockStreamSpec), intermediateStreamImpl.getOperatorSpec());
-    assertEquals(graph.getOutputStreams().get(mockStreamSpec), intermediateStreamImpl.getOutputStream());
-    assertEquals(mockStreamSpec, intermediateStreamImpl.getStreamSpec());
-    assertTrue(intermediateStreamImpl.getOutputStream().getKeySerde() instanceof NoOpSerde);
-    assertEquals(mockValueSerde, intermediateStreamImpl.getOutputStream().getValueSerde());
-    assertTrue(((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getKeySerde() instanceof NoOpSerde);
-    assertEquals(mockValueSerde, ((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getValueSerde());
-  }
-
-  @Test
-  public void testGetIntermediateStreamWithKeyValueSerde() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    Config mockConfig = mock(Config.class);
-    StreamSpec mockStreamSpec = mock(StreamSpec.class);
-    String mockStreamName = "mockStreamName";
-    when(mockRunner.getStreamSpec(mockStreamName)).thenReturn(mockStreamSpec);
-
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mockConfig);
-
-    KVSerde mockKVSerde = mock(KVSerde.class);
-    Serde mockKeySerde = mock(Serde.class);
-    Serde mockValueSerde = mock(Serde.class);
-    doReturn(mockKeySerde).when(mockKVSerde).getKeySerde();
-    doReturn(mockValueSerde).when(mockKVSerde).getValueSerde();
-    IntermediateMessageStreamImpl<TestMessageEnvelope> intermediateStreamImpl =
-        graph.getIntermediateStream(mockStreamName, mockKVSerde);
-
-    assertEquals(graph.getInputOperators().get(mockStreamSpec), intermediateStreamImpl.getOperatorSpec());
-    assertEquals(graph.getOutputStreams().get(mockStreamSpec), intermediateStreamImpl.getOutputStream());
-    assertEquals(mockStreamSpec, intermediateStreamImpl.getStreamSpec());
-    assertEquals(mockKeySerde, intermediateStreamImpl.getOutputStream().getKeySerde());
-    assertEquals(mockValueSerde, intermediateStreamImpl.getOutputStream().getValueSerde());
-    assertEquals(mockKeySerde, ((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getKeySerde());
-    assertEquals(mockValueSerde, ((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getValueSerde());
-  }
-
-  @Test
-  public void testGetIntermediateStreamWithDefaultValueSerde() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    Config mockConfig = mock(Config.class);
-    StreamSpec mockStreamSpec = mock(StreamSpec.class);
-    String mockStreamName = "mockStreamName";
-    when(mockRunner.getStreamSpec(mockStreamName)).thenReturn(mockStreamSpec);
-
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mockConfig);
-
-    Serde mockValueSerde = mock(Serde.class);
-    graph.setDefaultSerde(mockValueSerde);
-    IntermediateMessageStreamImpl<TestMessageEnvelope> intermediateStreamImpl =
-        graph.getIntermediateStream(mockStreamName, null);
-
-    assertEquals(graph.getInputOperators().get(mockStreamSpec), intermediateStreamImpl.getOperatorSpec());
-    assertEquals(graph.getOutputStreams().get(mockStreamSpec), intermediateStreamImpl.getOutputStream());
-    assertEquals(mockStreamSpec, intermediateStreamImpl.getStreamSpec());
-    assertTrue(intermediateStreamImpl.getOutputStream().getKeySerde() instanceof NoOpSerde);
-    assertEquals(mockValueSerde, intermediateStreamImpl.getOutputStream().getValueSerde());
-    assertTrue(((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getKeySerde() instanceof NoOpSerde);
-    assertEquals(mockValueSerde, ((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getValueSerde());
-  }
-
-  @Test
-  public void testGetIntermediateStreamWithDefaultKeyValueSerde() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    Config mockConfig = mock(Config.class);
-    StreamSpec mockStreamSpec = mock(StreamSpec.class);
-    String mockStreamName = "mockStreamName";
-    when(mockRunner.getStreamSpec(mockStreamName)).thenReturn(mockStreamSpec);
-
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mockConfig);
-
-    KVSerde mockKVSerde = mock(KVSerde.class);
-    Serde mockKeySerde = mock(Serde.class);
-    Serde mockValueSerde = mock(Serde.class);
-    doReturn(mockKeySerde).when(mockKVSerde).getKeySerde();
-    doReturn(mockValueSerde).when(mockKVSerde).getValueSerde();
-    graph.setDefaultSerde(mockKVSerde);
-    IntermediateMessageStreamImpl<TestMessageEnvelope> intermediateStreamImpl =
-        graph.getIntermediateStream(mockStreamName, null);
-
-    assertEquals(graph.getInputOperators().get(mockStreamSpec), intermediateStreamImpl.getOperatorSpec());
-    assertEquals(graph.getOutputStreams().get(mockStreamSpec), intermediateStreamImpl.getOutputStream());
-    assertEquals(mockStreamSpec, intermediateStreamImpl.getStreamSpec());
-    assertEquals(mockKeySerde, intermediateStreamImpl.getOutputStream().getKeySerde());
-    assertEquals(mockValueSerde, intermediateStreamImpl.getOutputStream().getValueSerde());
-    assertEquals(mockKeySerde, ((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getKeySerde());
-    assertEquals(mockValueSerde, ((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getValueSerde());
-  }
-
-  @Test
-  public void testGetIntermediateStreamWithDefaultDefaultSerde() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    Config mockConfig = mock(Config.class);
-    StreamSpec mockStreamSpec = mock(StreamSpec.class);
-    String mockStreamName = "mockStreamName";
-    when(mockRunner.getStreamSpec(mockStreamName)).thenReturn(mockStreamSpec);
-
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mockConfig);
-    IntermediateMessageStreamImpl<TestMessageEnvelope> intermediateStreamImpl =
-        graph.getIntermediateStream(mockStreamName, null);
-
-    assertEquals(graph.getInputOperators().get(mockStreamSpec), intermediateStreamImpl.getOperatorSpec());
-    assertEquals(graph.getOutputStreams().get(mockStreamSpec), intermediateStreamImpl.getOutputStream());
-    assertEquals(mockStreamSpec, intermediateStreamImpl.getStreamSpec());
-    assertTrue(intermediateStreamImpl.getOutputStream().getKeySerde() instanceof NoOpSerde);
-    assertTrue(intermediateStreamImpl.getOutputStream().getValueSerde() instanceof NoOpSerde);
-    assertTrue(((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getKeySerde() instanceof NoOpSerde);
-    assertTrue(((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getValueSerde() instanceof NoOpSerde);
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testGetSameIntermediateStreamTwice() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mock(StreamSpec.class));
-
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mock(Config.class));
-    graph.getIntermediateStream("test-stream-1", mock(Serde.class));
-    graph.getIntermediateStream("test-stream-1", mock(Serde.class));
-  }
-
-  @Test
-  public void testGetNextOpIdIncrementsId() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    Config mockConfig = mock(Config.class);
-    when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName");
-    when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234");
-
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mockConfig);
-    assertEquals("jobName-1234-merge-0", graph.getNextOpId(OpCode.MERGE, null));
-    assertEquals("jobName-1234-join-customName", graph.getNextOpId(OpCode.JOIN, "customName"));
-    assertEquals("jobName-1234-map-2", graph.getNextOpId(OpCode.MAP, null));
-  }
-
-  @Test(expected = SamzaException.class)
-  public void testGetNextOpIdRejectsDuplicates() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    Config mockConfig = mock(Config.class);
-    when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName");
-    when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234");
-
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mockConfig);
-    assertEquals("jobName-1234-join-customName", graph.getNextOpId(OpCode.JOIN, "customName"));
-    graph.getNextOpId(OpCode.JOIN, "customName"); // should throw
-  }
-
-  @Test
-  public void testUserDefinedIdValidation() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    Config mockConfig = mock(Config.class);
-    when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName");
-    when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234");
-
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mockConfig);
-
-    // null and empty userDefinedIDs should fall back to autogenerated IDs.
-    try {
-      graph.getNextOpId(OpCode.FILTER, null);
-      graph.getNextOpId(OpCode.FILTER, "");
-      graph.getNextOpId(OpCode.FILTER, " ");
-      graph.getNextOpId(OpCode.FILTER, "\t");
-    } catch (SamzaException e) {
-      Assert.fail("Received an error with a null or empty operator ID instead of defaulting to auto-generated ID.");
-    }
-
-    List<String> validOpIds = ImmutableList.of("op.id", "op_id", "op-id", "1000", "op_1", "OP_ID");
-    for (String validOpId: validOpIds) {
-      try {
-        graph.getNextOpId(OpCode.FILTER, validOpId);
-      } catch (Exception e) {
-        Assert.fail("Received an exception with a valid operator ID: " + validOpId);
-      }
-    }
-
-    List<String> invalidOpIds = ImmutableList.of("op id", "op#id");
-    for (String invalidOpId: invalidOpIds) {
-      try {
-        graph.getNextOpId(OpCode.FILTER, invalidOpId);
-        Assert.fail("Did not receive an exception with an invalid operator ID: " + invalidOpId);
-      } catch (SamzaException e) { }
-    }
-  }
-
-  @Test
-  public void testGetInputStreamPreservesInsertionOrder() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    Config mockConfig = mock(Config.class);
-
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mockConfig);
-
-    StreamSpec testStreamSpec1 = new StreamSpec("test-stream-1", "physical-stream-1", "test-system");
-    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(testStreamSpec1);
-
-    StreamSpec testStreamSpec2 = new StreamSpec("test-stream-2", "physical-stream-2", "test-system");
-    when(mockRunner.getStreamSpec("test-stream-2")).thenReturn(testStreamSpec2);
-
-    StreamSpec testStreamSpec3 = new StreamSpec("test-stream-3", "physical-stream-3", "test-system");
-    when(mockRunner.getStreamSpec("test-stream-3")).thenReturn(testStreamSpec3);
-
-    graph.getInputStream("test-stream-1");
-    graph.getInputStream("test-stream-2");
-    graph.getInputStream("test-stream-3");
-
-    List<InputOperatorSpec> inputSpecs = new ArrayList<>(graph.getInputOperators().values());
-    Assert.assertEquals(inputSpecs.size(), 3);
-    Assert.assertEquals(inputSpecs.get(0).getStreamSpec(), testStreamSpec1);
-    Assert.assertEquals(inputSpecs.get(1).getStreamSpec(), testStreamSpec2);
-    Assert.assertEquals(inputSpecs.get(2).getStreamSpec(), testStreamSpec3);
-  }
-
-  @Test
-  public void testGetTable() {
-    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    Config mockConfig = mock(Config.class);
-    StreamGraphImpl graph = new StreamGraphImpl(mockRunner, mockConfig);
-
-    BaseTableDescriptor mockTableDescriptor = mock(BaseTableDescriptor.class);
-    when(mockTableDescriptor.getTableSpec()).thenReturn(
-        new TableSpec("t1", KVSerde.of(new NoOpSerde(), new NoOpSerde()), "", new HashMap<>()));
-    Assert.assertNotNull(graph.getTable(mockTableDescriptor));
-  }
-}


[06/10] samza git commit: SAMZA-1659: Serializable OperatorSpec

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphSpec.java b/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphSpec.java
new file mode 100644
index 0000000..e476abc
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/operators/TestStreamGraphSpec.java
@@ -0,0 +1,601 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for THE
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators;
+
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.operators.data.TestMessageEnvelope;
+import org.apache.samza.operators.spec.InputOperatorSpec;
+import org.apache.samza.operators.spec.OperatorSpec.OpCode;
+import org.apache.samza.operators.spec.OutputStreamImpl;
+import org.apache.samza.operators.stream.IntermediateMessageStreamImpl;
+import org.apache.samza.runtime.ApplicationRunner;
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.serializers.NoOpSerde;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.system.StreamSpec;
+import org.apache.samza.table.TableSpec;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class TestStreamGraphSpec {
+
+  @Test
+  public void testGetInputStreamWithValueSerde() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    StreamSpec mockStreamSpec = mock(StreamSpec.class);
+    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
+
+    Serde mockValueSerde = mock(Serde.class);
+    MessageStream<TestMessageEnvelope> inputStream = graphSpec.getInputStream("test-stream-1", mockValueSerde);
+
+    InputOperatorSpec<String, TestMessageEnvelope> inputOpSpec =
+        (InputOperatorSpec) ((MessageStreamImpl<TestMessageEnvelope>) inputStream).getOperatorSpec();
+    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
+    assertEquals(graphSpec.getInputOperators().get(mockStreamSpec), inputOpSpec);
+    assertEquals(mockStreamSpec, inputOpSpec.getStreamSpec());
+    assertTrue(inputOpSpec.getKeySerde() instanceof NoOpSerde);
+    assertEquals(mockValueSerde, inputOpSpec.getValueSerde());
+  }
+
+  @Test
+  public void testGetInputStreamWithKeyValueSerde() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    StreamSpec mockStreamSpec = mock(StreamSpec.class);
+    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
+
+    KVSerde mockKVSerde = mock(KVSerde.class);
+    Serde mockKeySerde = mock(Serde.class);
+    Serde mockValueSerde = mock(Serde.class);
+    doReturn(mockKeySerde).when(mockKVSerde).getKeySerde();
+    doReturn(mockValueSerde).when(mockKVSerde).getValueSerde();
+    MessageStream<TestMessageEnvelope> inputStream = graphSpec.getInputStream("test-stream-1", mockKVSerde);
+
+    InputOperatorSpec<String, TestMessageEnvelope> inputOpSpec =
+        (InputOperatorSpec) ((MessageStreamImpl<TestMessageEnvelope>) inputStream).getOperatorSpec();
+    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
+    assertEquals(graphSpec.getInputOperators().get(mockStreamSpec), inputOpSpec);
+    assertEquals(mockStreamSpec, inputOpSpec.getStreamSpec());
+    assertEquals(mockKeySerde, inputOpSpec.getKeySerde());
+    assertEquals(mockValueSerde, inputOpSpec.getValueSerde());
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testGetInputStreamWithNullSerde() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    StreamSpec mockStreamSpec = mock(StreamSpec.class);
+    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
+
+    graphSpec.getInputStream("test-stream-1", null);
+  }
+
+  @Test
+  public void testGetInputStreamWithDefaultValueSerde() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    StreamSpec mockStreamSpec = mock(StreamSpec.class);
+    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
+
+    Serde mockValueSerde = mock(Serde.class);
+    graphSpec.setDefaultSerde(mockValueSerde);
+    MessageStream<TestMessageEnvelope> inputStream = graphSpec.getInputStream("test-stream-1");
+
+    InputOperatorSpec<String, TestMessageEnvelope> inputOpSpec =
+        (InputOperatorSpec) ((MessageStreamImpl<TestMessageEnvelope>) inputStream).getOperatorSpec();
+    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
+    assertEquals(graphSpec.getInputOperators().get(mockStreamSpec), inputOpSpec);
+    assertEquals(mockStreamSpec, inputOpSpec.getStreamSpec());
+    assertTrue(inputOpSpec.getKeySerde() instanceof NoOpSerde);
+    assertEquals(mockValueSerde, inputOpSpec.getValueSerde());
+  }
+
+  @Test
+  public void testGetInputStreamWithDefaultKeyValueSerde() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    StreamSpec mockStreamSpec = mock(StreamSpec.class);
+    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
+
+    KVSerde mockKVSerde = mock(KVSerde.class);
+    Serde mockKeySerde = mock(Serde.class);
+    Serde mockValueSerde = mock(Serde.class);
+    doReturn(mockKeySerde).when(mockKVSerde).getKeySerde();
+    doReturn(mockValueSerde).when(mockKVSerde).getValueSerde();
+    graphSpec.setDefaultSerde(mockKVSerde);
+    MessageStream<TestMessageEnvelope> inputStream = graphSpec.getInputStream("test-stream-1");
+
+    InputOperatorSpec<String, TestMessageEnvelope> inputOpSpec =
+        (InputOperatorSpec) ((MessageStreamImpl<TestMessageEnvelope>) inputStream).getOperatorSpec();
+    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
+    assertEquals(graphSpec.getInputOperators().get(mockStreamSpec), inputOpSpec);
+    assertEquals(mockStreamSpec, inputOpSpec.getStreamSpec());
+    assertEquals(mockKeySerde, inputOpSpec.getKeySerde());
+    assertEquals(mockValueSerde, inputOpSpec.getValueSerde());
+  }
+
+  @Test
+  public void testGetInputStreamWithDefaultDefaultSerde() {
+    // default default serde == user hasn't provided a default serde
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    StreamSpec mockStreamSpec = mock(StreamSpec.class);
+    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
+
+    MessageStream<TestMessageEnvelope> inputStream = graphSpec.getInputStream("test-stream-1");
+
+    InputOperatorSpec<String, TestMessageEnvelope> inputOpSpec =
+        (InputOperatorSpec) ((MessageStreamImpl<TestMessageEnvelope>) inputStream).getOperatorSpec();
+    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
+    assertEquals(graphSpec.getInputOperators().get(mockStreamSpec), inputOpSpec);
+    assertEquals(mockStreamSpec, inputOpSpec.getStreamSpec());
+    assertTrue(inputOpSpec.getKeySerde() instanceof NoOpSerde);
+    assertTrue(inputOpSpec.getValueSerde() instanceof NoOpSerde);
+  }
+
+  @Test
+  public void testGetInputStreamWithRelaxedTypes() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    StreamSpec mockStreamSpec = mock(StreamSpec.class);
+    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
+
+    MessageStream<TestMessageEnvelope> inputStream = graphSpec.getInputStream("test-stream-1");
+
+    InputOperatorSpec<String, TestMessageEnvelope> inputOpSpec =
+        (InputOperatorSpec) ((MessageStreamImpl<TestMessageEnvelope>) inputStream).getOperatorSpec();
+    assertEquals(OpCode.INPUT, inputOpSpec.getOpCode());
+    assertEquals(graphSpec.getInputOperators().get(mockStreamSpec), inputOpSpec);
+    assertEquals(mockStreamSpec, inputOpSpec.getStreamSpec());
+  }
+
+  @Test
+  public void testMultipleGetInputStreams() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    StreamSpec mockStreamSpec1 = mock(StreamSpec.class);
+    StreamSpec mockStreamSpec2 = mock(StreamSpec.class);
+    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec1);
+    when(mockRunner.getStreamSpec("test-stream-2")).thenReturn(mockStreamSpec2);
+
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
+    MessageStream<Object> inputStream1 = graphSpec.getInputStream("test-stream-1");
+    MessageStream<Object> inputStream2 = graphSpec.getInputStream("test-stream-2");
+
+    InputOperatorSpec<String, TestMessageEnvelope> inputOpSpec1 =
+        (InputOperatorSpec) ((MessageStreamImpl<Object>) inputStream1).getOperatorSpec();
+    InputOperatorSpec<String, TestMessageEnvelope> inputOpSpec2 =
+        (InputOperatorSpec) ((MessageStreamImpl<Object>) inputStream2).getOperatorSpec();
+
+    assertEquals(graphSpec.getInputOperators().size(), 2);
+    assertEquals(graphSpec.getInputOperators().get(mockStreamSpec1), inputOpSpec1);
+    assertEquals(graphSpec.getInputOperators().get(mockStreamSpec2), inputOpSpec2);
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testGetSameInputStreamTwice() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mock(StreamSpec.class));
+
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
+    graphSpec.getInputStream("test-stream-1");
+    // should throw exception
+    graphSpec.getInputStream("test-stream-1");
+  }
+
+  @Test
+  public void testGetOutputStreamWithValueSerde() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    StreamSpec mockStreamSpec = mock(StreamSpec.class);
+    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
+
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
+
+    Serde mockValueSerde = mock(Serde.class);
+    OutputStream<TestMessageEnvelope> outputStream =
+        graphSpec.getOutputStream("test-stream-1", mockValueSerde);
+
+    OutputStreamImpl<TestMessageEnvelope> outputStreamImpl = (OutputStreamImpl) outputStream;
+    assertEquals(graphSpec.getOutputStreams().get(mockStreamSpec), outputStreamImpl);
+    assertEquals(mockStreamSpec, outputStreamImpl.getStreamSpec());
+    assertTrue(outputStreamImpl.getKeySerde() instanceof NoOpSerde);
+    assertEquals(mockValueSerde, outputStreamImpl.getValueSerde());
+  }
+
+  @Test
+  public void testGetOutputStreamWithKeyValueSerde() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    StreamSpec mockStreamSpec = mock(StreamSpec.class);
+    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
+
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
+    KVSerde mockKVSerde = mock(KVSerde.class);
+    Serde mockKeySerde = mock(Serde.class);
+    Serde mockValueSerde = mock(Serde.class);
+    doReturn(mockKeySerde).when(mockKVSerde).getKeySerde();
+    doReturn(mockValueSerde).when(mockKVSerde).getValueSerde();
+    graphSpec.setDefaultSerde(mockKVSerde);
+    OutputStream<TestMessageEnvelope> outputStream = graphSpec.getOutputStream("test-stream-1", mockKVSerde);
+
+    OutputStreamImpl<TestMessageEnvelope> outputStreamImpl = (OutputStreamImpl) outputStream;
+    assertEquals(graphSpec.getOutputStreams().get(mockStreamSpec), outputStreamImpl);
+    assertEquals(mockStreamSpec, outputStreamImpl.getStreamSpec());
+    assertEquals(mockKeySerde, outputStreamImpl.getKeySerde());
+    assertEquals(mockValueSerde, outputStreamImpl.getValueSerde());
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testGetOutputStreamWithNullSerde() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    StreamSpec mockStreamSpec = mock(StreamSpec.class);
+    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
+
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
+
+    graphSpec.getOutputStream("test-stream-1", null);
+  }
+
+  @Test
+  public void testGetOutputStreamWithDefaultValueSerde() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    StreamSpec mockStreamSpec = mock(StreamSpec.class);
+    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
+
+    Serde mockValueSerde = mock(Serde.class);
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
+    graphSpec.setDefaultSerde(mockValueSerde);
+    OutputStream<TestMessageEnvelope> outputStream = graphSpec.getOutputStream("test-stream-1");
+
+    OutputStreamImpl<TestMessageEnvelope> outputStreamImpl = (OutputStreamImpl) outputStream;
+    assertEquals(graphSpec.getOutputStreams().get(mockStreamSpec), outputStreamImpl);
+    assertEquals(mockStreamSpec, outputStreamImpl.getStreamSpec());
+    assertTrue(outputStreamImpl.getKeySerde() instanceof NoOpSerde);
+    assertEquals(mockValueSerde, outputStreamImpl.getValueSerde());
+  }
+
+  @Test
+  public void testGetOutputStreamWithDefaultKeyValueSerde() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    StreamSpec mockStreamSpec = mock(StreamSpec.class);
+    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
+
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
+    KVSerde mockKVSerde = mock(KVSerde.class);
+    Serde mockKeySerde = mock(Serde.class);
+    Serde mockValueSerde = mock(Serde.class);
+    doReturn(mockKeySerde).when(mockKVSerde).getKeySerde();
+    doReturn(mockValueSerde).when(mockKVSerde).getValueSerde();
+    graphSpec.setDefaultSerde(mockKVSerde);
+
+    OutputStream<TestMessageEnvelope> outputStream = graphSpec.getOutputStream("test-stream-1");
+
+    OutputStreamImpl<TestMessageEnvelope> outputStreamImpl = (OutputStreamImpl) outputStream;
+    assertEquals(graphSpec.getOutputStreams().get(mockStreamSpec), outputStreamImpl);
+    assertEquals(mockStreamSpec, outputStreamImpl.getStreamSpec());
+    assertEquals(mockKeySerde, outputStreamImpl.getKeySerde());
+    assertEquals(mockValueSerde, outputStreamImpl.getValueSerde());
+  }
+
+  @Test
+  public void testGetOutputStreamWithDefaultDefaultSerde() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    StreamSpec mockStreamSpec = mock(StreamSpec.class);
+    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mockStreamSpec);
+
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
+
+    OutputStream<TestMessageEnvelope> outputStream = graphSpec.getOutputStream("test-stream-1");
+
+    OutputStreamImpl<TestMessageEnvelope> outputStreamImpl = (OutputStreamImpl) outputStream;
+    assertEquals(graphSpec.getOutputStreams().get(mockStreamSpec), outputStreamImpl);
+    assertEquals(mockStreamSpec, outputStreamImpl.getStreamSpec());
+    assertTrue(outputStreamImpl.getKeySerde() instanceof NoOpSerde);
+    assertTrue(outputStreamImpl.getValueSerde() instanceof NoOpSerde);
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testSetDefaultSerdeAfterGettingStreams() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mock(StreamSpec.class));
+
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
+    graphSpec.getInputStream("test-stream-1");
+    graphSpec.setDefaultSerde(mock(Serde.class)); // should throw exception
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testSetDefaultSerdeAfterGettingOutputStream() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mock(StreamSpec.class));
+
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
+    graphSpec.getOutputStream("test-stream-1");
+    graphSpec.setDefaultSerde(mock(Serde.class)); // should throw exception
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testSetDefaultSerdeAfterGettingIntermediateStream() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mock(StreamSpec.class));
+
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
+    graphSpec.getIntermediateStream("test-stream-1", null);
+    graphSpec.setDefaultSerde(mock(Serde.class)); // should throw exception
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testGetSameOutputStreamTwice() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mock(StreamSpec.class));
+
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
+    graphSpec.getOutputStream("test-stream-1");
+    graphSpec.getOutputStream("test-stream-1"); // should throw exception
+  }
+
+  @Test
+  public void testGetIntermediateStreamWithValueSerde() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    Config mockConfig = mock(Config.class);
+    StreamSpec mockStreamSpec = mock(StreamSpec.class);
+    String mockStreamName = "mockStreamName";
+    when(mockRunner.getStreamSpec(mockStreamName)).thenReturn(mockStreamSpec);
+
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig);
+
+    Serde mockValueSerde = mock(Serde.class);
+    IntermediateMessageStreamImpl<TestMessageEnvelope> intermediateStreamImpl =
+        graphSpec.getIntermediateStream(mockStreamName, mockValueSerde);
+
+    assertEquals(graphSpec.getInputOperators().get(mockStreamSpec), intermediateStreamImpl.getOperatorSpec());
+    assertEquals(graphSpec.getOutputStreams().get(mockStreamSpec), intermediateStreamImpl.getOutputStream());
+    assertEquals(mockStreamSpec, intermediateStreamImpl.getStreamSpec());
+    assertTrue(intermediateStreamImpl.getOutputStream().getKeySerde() instanceof NoOpSerde);
+    assertEquals(mockValueSerde, intermediateStreamImpl.getOutputStream().getValueSerde());
+    assertTrue(((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getKeySerde() instanceof NoOpSerde);
+    assertEquals(mockValueSerde, ((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getValueSerde());
+  }
+
+  @Test
+  public void testGetIntermediateStreamWithKeyValueSerde() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    Config mockConfig = mock(Config.class);
+    StreamSpec mockStreamSpec = mock(StreamSpec.class);
+    String mockStreamName = "mockStreamName";
+    when(mockRunner.getStreamSpec(mockStreamName)).thenReturn(mockStreamSpec);
+
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig);
+
+    KVSerde mockKVSerde = mock(KVSerde.class);
+    Serde mockKeySerde = mock(Serde.class);
+    Serde mockValueSerde = mock(Serde.class);
+    doReturn(mockKeySerde).when(mockKVSerde).getKeySerde();
+    doReturn(mockValueSerde).when(mockKVSerde).getValueSerde();
+    IntermediateMessageStreamImpl<TestMessageEnvelope> intermediateStreamImpl =
+        graphSpec.getIntermediateStream(mockStreamName, mockKVSerde);
+
+    assertEquals(graphSpec.getInputOperators().get(mockStreamSpec), intermediateStreamImpl.getOperatorSpec());
+    assertEquals(graphSpec.getOutputStreams().get(mockStreamSpec), intermediateStreamImpl.getOutputStream());
+    assertEquals(mockStreamSpec, intermediateStreamImpl.getStreamSpec());
+    assertEquals(mockKeySerde, intermediateStreamImpl.getOutputStream().getKeySerde());
+    assertEquals(mockValueSerde, intermediateStreamImpl.getOutputStream().getValueSerde());
+    assertEquals(mockKeySerde, ((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getKeySerde());
+    assertEquals(mockValueSerde, ((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getValueSerde());
+  }
+
+  @Test
+  public void testGetIntermediateStreamWithDefaultValueSerde() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    Config mockConfig = mock(Config.class);
+    StreamSpec mockStreamSpec = mock(StreamSpec.class);
+    String mockStreamName = "mockStreamName";
+    when(mockRunner.getStreamSpec(mockStreamName)).thenReturn(mockStreamSpec);
+
+    StreamGraphSpec graph = new StreamGraphSpec(mockRunner, mockConfig);
+
+    Serde mockValueSerde = mock(Serde.class);
+    graph.setDefaultSerde(mockValueSerde);
+    IntermediateMessageStreamImpl<TestMessageEnvelope> intermediateStreamImpl =
+        graph.getIntermediateStream(mockStreamName, null);
+
+    assertEquals(graph.getInputOperators().get(mockStreamSpec), intermediateStreamImpl.getOperatorSpec());
+    assertEquals(graph.getOutputStreams().get(mockStreamSpec), intermediateStreamImpl.getOutputStream());
+    assertEquals(mockStreamSpec, intermediateStreamImpl.getStreamSpec());
+    assertTrue(intermediateStreamImpl.getOutputStream().getKeySerde() instanceof NoOpSerde);
+    assertEquals(mockValueSerde, intermediateStreamImpl.getOutputStream().getValueSerde());
+    assertTrue(((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getKeySerde() instanceof NoOpSerde);
+    assertEquals(mockValueSerde, ((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getValueSerde());
+  }
+
+  @Test
+  public void testGetIntermediateStreamWithDefaultKeyValueSerde() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    Config mockConfig = mock(Config.class);
+    StreamSpec mockStreamSpec = mock(StreamSpec.class);
+    String mockStreamName = "mockStreamName";
+    when(mockRunner.getStreamSpec(mockStreamName)).thenReturn(mockStreamSpec);
+
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig);
+
+    KVSerde mockKVSerde = mock(KVSerde.class);
+    Serde mockKeySerde = mock(Serde.class);
+    Serde mockValueSerde = mock(Serde.class);
+    doReturn(mockKeySerde).when(mockKVSerde).getKeySerde();
+    doReturn(mockValueSerde).when(mockKVSerde).getValueSerde();
+    graphSpec.setDefaultSerde(mockKVSerde);
+    IntermediateMessageStreamImpl<TestMessageEnvelope> intermediateStreamImpl =
+        graphSpec.getIntermediateStream(mockStreamName, null);
+
+    assertEquals(graphSpec.getInputOperators().get(mockStreamSpec), intermediateStreamImpl.getOperatorSpec());
+    assertEquals(graphSpec.getOutputStreams().get(mockStreamSpec), intermediateStreamImpl.getOutputStream());
+    assertEquals(mockStreamSpec, intermediateStreamImpl.getStreamSpec());
+    assertEquals(mockKeySerde, intermediateStreamImpl.getOutputStream().getKeySerde());
+    assertEquals(mockValueSerde, intermediateStreamImpl.getOutputStream().getValueSerde());
+    assertEquals(mockKeySerde, ((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getKeySerde());
+    assertEquals(mockValueSerde, ((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getValueSerde());
+  }
+
+  @Test
+  public void testGetIntermediateStreamWithDefaultDefaultSerde() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    Config mockConfig = mock(Config.class);
+    StreamSpec mockStreamSpec = mock(StreamSpec.class);
+    String mockStreamName = "mockStreamName";
+    when(mockRunner.getStreamSpec(mockStreamName)).thenReturn(mockStreamSpec);
+
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig);
+    IntermediateMessageStreamImpl<TestMessageEnvelope> intermediateStreamImpl =
+        graphSpec.getIntermediateStream(mockStreamName, null);
+
+    assertEquals(graphSpec.getInputOperators().get(mockStreamSpec), intermediateStreamImpl.getOperatorSpec());
+    assertEquals(graphSpec.getOutputStreams().get(mockStreamSpec), intermediateStreamImpl.getOutputStream());
+    assertEquals(mockStreamSpec, intermediateStreamImpl.getStreamSpec());
+    assertTrue(intermediateStreamImpl.getOutputStream().getKeySerde() instanceof NoOpSerde);
+    assertTrue(intermediateStreamImpl.getOutputStream().getValueSerde() instanceof NoOpSerde);
+    assertTrue(((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getKeySerde() instanceof NoOpSerde);
+    assertTrue(((InputOperatorSpec) intermediateStreamImpl.getOperatorSpec()).getValueSerde() instanceof NoOpSerde);
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testGetSameIntermediateStreamTwice() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(mock(StreamSpec.class));
+
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
+    graphSpec.getIntermediateStream("test-stream-1", mock(Serde.class));
+    graphSpec.getIntermediateStream("test-stream-1", mock(Serde.class));
+  }
+
+  @Test
+  public void testGetNextOpIdIncrementsId() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    Config mockConfig = mock(Config.class);
+    when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName");
+    when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234");
+
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig);
+    assertEquals("jobName-1234-merge-0", graphSpec.getNextOpId(OpCode.MERGE, null));
+    assertEquals("jobName-1234-join-customName", graphSpec.getNextOpId(OpCode.JOIN, "customName"));
+    assertEquals("jobName-1234-map-2", graphSpec.getNextOpId(OpCode.MAP, null));
+  }
+
+  @Test(expected = SamzaException.class)
+  public void testGetNextOpIdRejectsDuplicates() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    Config mockConfig = mock(Config.class);
+    when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName");
+    when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234");
+
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig);
+    assertEquals("jobName-1234-join-customName", graphSpec.getNextOpId(OpCode.JOIN, "customName"));
+    graphSpec.getNextOpId(OpCode.JOIN, "customName"); // should throw
+  }
+
+  @Test
+  public void testUserDefinedIdValidation() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    Config mockConfig = mock(Config.class);
+    when(mockConfig.get(eq(JobConfig.JOB_NAME()))).thenReturn("jobName");
+    when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("1234");
+
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig);
+
+    // null and empty userDefinedIDs should fall back to autogenerated IDs.
+    try {
+      graphSpec.getNextOpId(OpCode.FILTER, null);
+      graphSpec.getNextOpId(OpCode.FILTER, "");
+      graphSpec.getNextOpId(OpCode.FILTER, " ");
+      graphSpec.getNextOpId(OpCode.FILTER, "\t");
+    } catch (SamzaException e) {
+      fail("Received an error with a null or empty operator ID instead of defaulting to auto-generated ID.");
+    }
+
+    List<String> validOpIds = ImmutableList.of("op.id", "op_id", "op-id", "1000", "op_1", "OP_ID");
+    for (String validOpId: validOpIds) {
+      try {
+        graphSpec.getNextOpId(OpCode.FILTER, validOpId);
+      } catch (Exception e) {
+        fail("Received an exception with a valid operator ID: " + validOpId);
+      }
+    }
+
+    List<String> invalidOpIds = ImmutableList.of("op id", "op#id");
+    for (String invalidOpId: invalidOpIds) {
+      try {
+        graphSpec.getNextOpId(OpCode.FILTER, invalidOpId);
+        fail("Did not receive an exception with an invalid operator ID: " + invalidOpId);
+      } catch (SamzaException e) { }
+    }
+  }
+
+  @Test
+  public void testGetInputStreamPreservesInsertionOrder() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    Config mockConfig = mock(Config.class);
+
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig);
+
+    StreamSpec testStreamSpec1 = new StreamSpec("test-stream-1", "physical-stream-1", "test-system");
+    when(mockRunner.getStreamSpec("test-stream-1")).thenReturn(testStreamSpec1);
+
+    StreamSpec testStreamSpec2 = new StreamSpec("test-stream-2", "physical-stream-2", "test-system");
+    when(mockRunner.getStreamSpec("test-stream-2")).thenReturn(testStreamSpec2);
+
+    StreamSpec testStreamSpec3 = new StreamSpec("test-stream-3", "physical-stream-3", "test-system");
+    when(mockRunner.getStreamSpec("test-stream-3")).thenReturn(testStreamSpec3);
+
+    graphSpec.getInputStream("test-stream-1");
+    graphSpec.getInputStream("test-stream-2");
+    graphSpec.getInputStream("test-stream-3");
+
+    List<InputOperatorSpec> inputSpecs = new ArrayList<>(graphSpec.getInputOperators().values());
+    assertEquals(inputSpecs.size(), 3);
+    assertEquals(inputSpecs.get(0).getStreamSpec(), testStreamSpec1);
+    assertEquals(inputSpecs.get(1).getStreamSpec(), testStreamSpec2);
+    assertEquals(inputSpecs.get(2).getStreamSpec(), testStreamSpec3);
+  }
+
+  @Test
+  public void testGetTable() {
+    ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+    Config mockConfig = mock(Config.class);
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig);
+
+    BaseTableDescriptor mockTableDescriptor = mock(BaseTableDescriptor.class);
+    when(mockTableDescriptor.getTableSpec()).thenReturn(
+        new TableSpec("t1", KVSerde.of(new NoOpSerde(), new NoOpSerde()), "", new HashMap<>()));
+    assertNotNull(graphSpec.getTable(mockTableDescriptor));
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/operators/data/TestOutputMessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/data/TestOutputMessageEnvelope.java b/samza-core/src/test/java/org/apache/samza/operators/data/TestOutputMessageEnvelope.java
index f9537a3..519e5df 100644
--- a/samza-core/src/test/java/org/apache/samza/operators/data/TestOutputMessageEnvelope.java
+++ b/samza-core/src/test/java/org/apache/samza/operators/data/TestOutputMessageEnvelope.java
@@ -35,5 +35,19 @@ public class TestOutputMessageEnvelope {
   public String getKey() {
     return this.key;
   }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof TestOutputMessageEnvelope)) {
+      return false;
+    }
+    TestOutputMessageEnvelope otherMsg = (TestOutputMessageEnvelope) other;
+    return this.key.equals(otherMsg.key) && this.value.equals(otherMsg.value);
+  }
+
+  @Override
+  public int hashCode() {
+    return String.format("%s:%d", key, value).hashCode();
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 2d8d1eb..b87e5ed 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
@@ -21,11 +21,17 @@ package org.apache.samza.operators.impl;
 
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.ArrayList;
 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 java.util.function.BiFunction;
+import java.util.function.Function;
 import org.apache.samza.Partition;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JobConfig;
@@ -39,9 +45,11 @@ 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.StreamGraphSpec;
 import org.apache.samza.operators.OutputStream;
-import org.apache.samza.operators.StreamGraphImpl;
+import org.apache.samza.operators.functions.ClosableFunction;
 import org.apache.samza.operators.functions.FilterFunction;
+import org.apache.samza.operators.functions.InitableFunction;
 import org.apache.samza.operators.functions.JoinFunction;
 import org.apache.samza.operators.functions.MapFunction;
 import org.apache.samza.operators.impl.store.TimestampedValue;
@@ -58,34 +66,160 @@ 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 java.util.List;
 import org.apache.samza.task.TaskCoordinator;
 import org.apache.samza.util.Clock;
 import org.apache.samza.util.SystemClock;
+import org.junit.After;
 import org.junit.Test;
 
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotSame;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 public class TestOperatorImplGraph {
 
+  private void addOperatorRecursively(HashSet<OperatorImpl> s, OperatorImpl op) {
+    List<OperatorImpl> operators = new ArrayList<>();
+    operators.add(op);
+    while (!operators.isEmpty()) {
+      OperatorImpl opImpl = operators.remove(0);
+      s.add(opImpl);
+      if (!opImpl.registeredOperators.isEmpty()) {
+        operators.addAll(opImpl.registeredOperators);
+      }
+    }
+  }
+
+  static class TestMapFunction<M, OM> extends BaseTestFunction implements MapFunction<M, OM> {
+    final Function<M, OM> mapFn;
+
+    public TestMapFunction(String opId, Function<M, OM> mapFn) {
+      super(opId);
+      this.mapFn = mapFn;
+    }
+
+    @Override
+    public OM apply(M message) {
+      return this.mapFn.apply(message);
+    }
+  }
+
+  static class TestJoinFunction<K, M, JM, RM> extends BaseTestFunction implements JoinFunction<K, M, JM, RM> {
+    final BiFunction<M, JM, RM> joiner;
+    final Function<M, K> firstKeyFn;
+    final Function<JM, K> secondKeyFn;
+    final Collection<RM> joinResults = new HashSet<>();
+
+    public TestJoinFunction(String opId, BiFunction<M, JM, RM> joiner, Function<M, K> firstKeyFn, Function<JM, K> secondKeyFn) {
+      super(opId);
+      this.joiner = joiner;
+      this.firstKeyFn = firstKeyFn;
+      this.secondKeyFn = secondKeyFn;
+    }
+
+    @Override
+    public RM apply(M message, JM otherMessage) {
+      RM result = this.joiner.apply(message, otherMessage);
+      this.joinResults.add(result);
+      return result;
+    }
+
+    @Override
+    public K getFirstKey(M message) {
+      return this.firstKeyFn.apply(message);
+    }
+
+    @Override
+    public K getSecondKey(JM message) {
+      return this.secondKeyFn.apply(message);
+    }
+  }
+
+  static abstract class BaseTestFunction implements InitableFunction, ClosableFunction, Serializable {
+
+    static Map<TaskName, Map<String, BaseTestFunction>> perTaskFunctionMap = new HashMap<>();
+    static Map<TaskName, List<String>> perTaskInitList = new HashMap<>();
+    static Map<TaskName, List<String>> perTaskCloseList = new HashMap<>();
+    int numInitCalled = 0;
+    int numCloseCalled = 0;
+    TaskName taskName = null;
+    final String opId;
+
+    public BaseTestFunction(String opId) {
+      this.opId = opId;
+    }
+
+    static public void reset() {
+      perTaskFunctionMap.clear();
+      perTaskCloseList.clear();
+      perTaskInitList.clear();
+    }
+
+    static public BaseTestFunction getInstanceByTaskName(TaskName taskName, String opId) {
+      return perTaskFunctionMap.get(taskName).get(opId);
+    }
+
+    static public List<String> getInitListByTaskName(TaskName taskName) {
+      return perTaskInitList.get(taskName);
+    }
+
+    static public List<String> getCloseListByTaskName(TaskName taskName) {
+      return perTaskCloseList.get(taskName);
+    }
+
+    @Override
+    public void close() {
+      if (this.taskName == null) {
+        throw new IllegalStateException("Close called before init");
+      }
+      if (perTaskFunctionMap.get(this.taskName) == null || !perTaskFunctionMap.get(this.taskName).containsKey(opId)) {
+        throw new IllegalStateException("Close called before init");
+      }
+
+      if (perTaskCloseList.get(this.taskName) == null) {
+        perTaskCloseList.put(taskName, new ArrayList<String>() { { this.add(opId); } });
+      } else {
+        perTaskCloseList.get(taskName).add(opId);
+      }
+
+      this.numCloseCalled++;
+    }
+
+    @Override
+    public void init(Config config, TaskContext context) {
+      if (perTaskFunctionMap.get(context.getTaskName()) == null) {
+        perTaskFunctionMap.put(context.getTaskName(), new HashMap<String, BaseTestFunction>() { { this.put(opId, BaseTestFunction.this); } });
+      } else {
+        if (perTaskFunctionMap.get(context.getTaskName()).containsKey(opId)) {
+          throw new IllegalStateException(String.format("Multiple init called for op %s in the same task instance %s", opId, this.taskName.getTaskName()));
+        }
+        perTaskFunctionMap.get(context.getTaskName()).put(opId, this);
+      }
+      if (perTaskInitList.get(context.getTaskName()) == null) {
+        perTaskInitList.put(context.getTaskName(), new ArrayList<String>() { { this.add(opId); } });
+      } else {
+        perTaskInitList.get(context.getTaskName()).add(opId);
+      }
+      this.taskName = context.getTaskName();
+      this.numInitCalled++;
+    }
+  }
+
+  @After
+  public void tearDown() {
+    BaseTestFunction.reset();
+  }
+
   @Test
   public void testEmptyChain() {
-    StreamGraphImpl streamGraph = new StreamGraphImpl(mock(ApplicationRunner.class), mock(Config.class));
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mock(ApplicationRunner.class), mock(Config.class));
     OperatorImplGraph opGraph =
-        new OperatorImplGraph(streamGraph, mock(Config.class), mock(TaskContextImpl.class), mock(Clock.class));
+        new OperatorImplGraph(graphSpec.getOperatorSpecGraph(), mock(Config.class), mock(TaskContextImpl.class), mock(Clock.class));
     assertEquals(0, opGraph.getAllInputOperators().size());
   }
 
@@ -94,10 +228,10 @@ public class TestOperatorImplGraph {
     ApplicationRunner mockRunner = mock(ApplicationRunner.class);
     when(mockRunner.getStreamSpec(eq("input"))).thenReturn(new StreamSpec("input", "input-stream", "input-system"));
     when(mockRunner.getStreamSpec(eq("output"))).thenReturn(mock(StreamSpec.class));
-    StreamGraphImpl streamGraph = new StreamGraphImpl(mockRunner, mock(Config.class));
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
 
-    MessageStream<Object> inputStream = streamGraph.getInputStream("input");
-    OutputStream<Object> outputStream = streamGraph.getOutputStream("output");
+    MessageStream<Object> inputStream = graphSpec.getInputStream("input");
+    OutputStream<Object> outputStream = graphSpec.getOutputStream("output");
 
     inputStream
         .filter(mock(FilterFunction.class))
@@ -108,7 +242,7 @@ public class TestOperatorImplGraph {
     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));
+        new OperatorImplGraph(graphSpec.getOperatorSpecGraph(), mock(Config.class), mockTaskContext, mock(Clock.class));
 
     InputOperatorImpl inputOpImpl = opImplGraph.getInputOperator(new SystemStream("input-system", "input-stream"));
     assertEquals(1, inputOpImpl.registeredOperators.size());
@@ -136,9 +270,9 @@ public class TestOperatorImplGraph {
     Config mockConfig = mock(Config.class);
     when(mockConfig.get(JobConfig.JOB_NAME())).thenReturn("jobName");
     when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("jobId");
-    StreamGraphImpl streamGraph = new StreamGraphImpl(mockRunner, mockConfig);
-    MessageStream<Object> inputStream = streamGraph.getInputStream("input");
-    OutputStream<KV<Integer, String>> outputStream = streamGraph
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig);
+    MessageStream<Object> inputStream = graphSpec.getInputStream("input");
+    OutputStream<KV<Integer, String>> outputStream = graphSpec
         .getOutputStream("output", KVSerde.of(mock(IntegerSerde.class), mock(StringSerde.class)));
 
     inputStream
@@ -160,7 +294,7 @@ public class TestOperatorImplGraph {
         new SamzaContainerContext("0", mockConfig, Collections.singleton(new TaskName("task 0")), new MetricsRegistryMap());
     when(mockTaskContext.getSamzaContainerContext()).thenReturn(containerContext);
     OperatorImplGraph opImplGraph =
-        new OperatorImplGraph(streamGraph, mockConfig, mockTaskContext, mock(Clock.class));
+        new OperatorImplGraph(graphSpec.getOperatorSpecGraph(), mockConfig, mockTaskContext, mock(Clock.class));
 
     InputOperatorImpl inputOpImpl = opImplGraph.getInputOperator(new SystemStream("input-system", "input-stream"));
     assertEquals(1, inputOpImpl.registeredOperators.size());
@@ -182,16 +316,16 @@ public class TestOperatorImplGraph {
   public void testBroadcastChain() {
     ApplicationRunner mockRunner = mock(ApplicationRunner.class);
     when(mockRunner.getStreamSpec(eq("input"))).thenReturn(new StreamSpec("input", "input-stream", "input-system"));
-    StreamGraphImpl streamGraph = new StreamGraphImpl(mockRunner, mock(Config.class));
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
 
-    MessageStream<Object> inputStream = streamGraph.getInputStream("input");
+    MessageStream<Object> inputStream = graphSpec.getInputStream("input");
     inputStream.filter(mock(FilterFunction.class));
     inputStream.map(mock(MapFunction.class));
 
     TaskContextImpl mockTaskContext = mock(TaskContextImpl.class);
     when(mockTaskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap());
     OperatorImplGraph opImplGraph =
-        new OperatorImplGraph(streamGraph, mock(Config.class), mockTaskContext, mock(Clock.class));
+        new OperatorImplGraph(graphSpec.getOperatorSpecGraph(), mock(Config.class), mockTaskContext, mock(Clock.class));
 
     InputOperatorImpl inputOpImpl = opImplGraph.getInputOperator(new SystemStream("input-system", "input-stream"));
     assertEquals(2, inputOpImpl.registeredOperators.size());
@@ -204,23 +338,36 @@ public class TestOperatorImplGraph {
   @Test
   public void testMergeChain() {
     ApplicationRunner mockRunner = mock(ApplicationRunner.class);
-    when(mockRunner.getStreamSpec(eq("input"))).thenReturn(new StreamSpec("input", "input-stream", "input-system"));
-    StreamGraphImpl streamGraph = new StreamGraphImpl(mockRunner, mock(Config.class));
+    when(mockRunner.getStreamSpec(eq("input")))
+        .thenReturn(new StreamSpec("input", "input-stream", "input-system"));
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mock(Config.class));
 
-    MessageStream<Object> inputStream = streamGraph.getInputStream("input");
+    MessageStream<Object> inputStream = graphSpec.getInputStream("input");
     MessageStream<Object> stream1 = inputStream.filter(mock(FilterFunction.class));
     MessageStream<Object> stream2 = inputStream.map(mock(MapFunction.class));
     MessageStream<Object> mergedStream = stream1.merge(Collections.singleton(stream2));
-    MapFunction mockMapFunction = mock(MapFunction.class);
-    mergedStream.map(mockMapFunction);
 
     TaskContextImpl mockTaskContext = mock(TaskContextImpl.class);
+    TaskName mockTaskName = mock(TaskName.class);
+    when(mockTaskContext.getTaskName()).thenReturn(mockTaskName);
     when(mockTaskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap());
+
+    MapFunction testMapFunction = new TestMapFunction<Object, Object>("test-map-1", (Function & Serializable) m -> m);
+    mergedStream.map(testMapFunction);
+
     OperatorImplGraph opImplGraph =
-        new OperatorImplGraph(streamGraph, mock(Config.class), mockTaskContext, mock(Clock.class));
+        new OperatorImplGraph(graphSpec.getOperatorSpecGraph(), mock(Config.class), mockTaskContext, mock(Clock.class));
+
+    Set<OperatorImpl> opSet = opImplGraph.getAllInputOperators().stream().collect(HashSet::new,
+        (s, op) -> addOperatorRecursively(s, op), HashSet::addAll);
+    Object[] mergeOps = opSet.stream().filter(op -> op.getOperatorSpec().getOpCode() == OpCode.MERGE).toArray();
+    assertEquals(mergeOps.length, 1);
+    assertEquals(((OperatorImpl) mergeOps[0]).registeredOperators.size(), 1);
+    OperatorImpl mapOp = (OperatorImpl) ((OperatorImpl) mergeOps[0]).registeredOperators.iterator().next();
+    assertEquals(mapOp.getOperatorSpec().getOpCode(), OpCode.MAP);
 
     // verify that the DAG after merge is only traversed & initialized once
-    verify(mockMapFunction, times(1)).init(any(Config.class), any(TaskContext.class));
+    assertEquals(TestMapFunction.getInstanceByTaskName(mockTaskName, "test-map-1").numInitCalled, 1);
   }
 
   @Test
@@ -231,25 +378,30 @@ public class TestOperatorImplGraph {
     Config mockConfig = mock(Config.class);
     when(mockConfig.get(JobConfig.JOB_NAME())).thenReturn("jobName");
     when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("jobId");
-    StreamGraphImpl streamGraph = new StreamGraphImpl(mockRunner, mockConfig);
-
-    JoinFunction mockJoinFunction = mock(JoinFunction.class);
-    MessageStream<Object> inputStream1 = streamGraph.getInputStream("input1", new NoOpSerde<>());
-    MessageStream<Object> inputStream2 = streamGraph.getInputStream("input2", new NoOpSerde<>());
-    inputStream1.join(inputStream2, mockJoinFunction,
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig);
+
+    Integer joinKey = new Integer(1);
+    Function<Object, Integer> keyFn = (Function & Serializable) m -> joinKey;
+    JoinFunction testJoinFunction = new TestJoinFunction("jobName-jobId-join-j1",
+        (BiFunction & Serializable) (m1, m2) -> KV.of(m1, m2), keyFn, keyFn);
+    MessageStream<Object> inputStream1 = graphSpec.getInputStream("input1", new NoOpSerde<>());
+    MessageStream<Object> inputStream2 = graphSpec.getInputStream("input2", new NoOpSerde<>());
+    inputStream1.join(inputStream2, testJoinFunction,
         mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(1), "j1");
 
+    TaskName mockTaskName = mock(TaskName.class);
     TaskContextImpl mockTaskContext = mock(TaskContextImpl.class);
+    when(mockTaskContext.getTaskName()).thenReturn(mockTaskName);
     when(mockTaskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap());
     KeyValueStore mockLeftStore = mock(KeyValueStore.class);
     when(mockTaskContext.getStore(eq("jobName-jobId-join-j1-L"))).thenReturn(mockLeftStore);
     KeyValueStore mockRightStore = mock(KeyValueStore.class);
     when(mockTaskContext.getStore(eq("jobName-jobId-join-j1-R"))).thenReturn(mockRightStore);
     OperatorImplGraph opImplGraph =
-        new OperatorImplGraph(streamGraph, mockConfig, mockTaskContext, mock(Clock.class));
+        new OperatorImplGraph(graphSpec.getOperatorSpecGraph(), mockConfig, mockTaskContext, mock(Clock.class));
 
     // verify that join function is initialized once.
-    verify(mockJoinFunction, times(1)).init(any(Config.class), any(TaskContext.class));
+    assertEquals(TestJoinFunction.getInstanceByTaskName(mockTaskName, "jobName-jobId-join-j1").numInitCalled, 1);
 
     InputOperatorImpl inputOpImpl1 = opImplGraph.getInputOperator(new SystemStream("input-system", "input-stream1"));
     InputOperatorImpl inputOpImpl2 = opImplGraph.getInputOperator(new SystemStream("input-system", "input-stream2"));
@@ -261,24 +413,23 @@ public class TestOperatorImplGraph {
     assertEquals(leftPartialJoinOpImpl.getOperatorSpec(), rightPartialJoinOpImpl.getOperatorSpec());
     assertNotSame(leftPartialJoinOpImpl, rightPartialJoinOpImpl);
 
-    Object joinKey = new Object();
     // verify that left partial join operator calls getFirstKey
     Object mockLeftMessage = mock(Object.class);
     long currentTimeMillis = System.currentTimeMillis();
     when(mockLeftStore.get(eq(joinKey))).thenReturn(new TimestampedValue<>(mockLeftMessage, currentTimeMillis));
-    when(mockJoinFunction.getFirstKey(eq(mockLeftMessage))).thenReturn(joinKey);
     inputOpImpl1.onMessage(KV.of("", mockLeftMessage), mock(MessageCollector.class), mock(TaskCoordinator.class));
-    verify(mockJoinFunction, times(1)).getFirstKey(mockLeftMessage);
 
     // verify that right partial join operator calls getSecondKey
     Object mockRightMessage = mock(Object.class);
     when(mockRightStore.get(eq(joinKey))).thenReturn(new TimestampedValue<>(mockRightMessage, currentTimeMillis));
-    when(mockJoinFunction.getSecondKey(eq(mockRightMessage))).thenReturn(joinKey);
     inputOpImpl2.onMessage(KV.of("", mockRightMessage), mock(MessageCollector.class), mock(TaskCoordinator.class));
-    verify(mockJoinFunction, times(1)).getSecondKey(mockRightMessage);
+
 
     // verify that the join function apply is called with the correct messages on match
-    verify(mockJoinFunction, times(1)).apply(mockLeftMessage, mockRightMessage);
+    assertEquals(((TestJoinFunction) TestJoinFunction.getInstanceByTaskName(mockTaskName, "jobName-jobId-join-j1")).joinResults.size(), 1);
+    KV joinResult = (KV) ((TestJoinFunction) TestJoinFunction.getInstanceByTaskName(mockTaskName, "jobName-jobId-join-j1")).joinResults.iterator().next();
+    assertEquals(joinResult.getKey(), mockLeftMessage);
+    assertEquals(joinResult.getValue(), mockRightMessage);
   }
 
   @Test
@@ -287,23 +438,25 @@ 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);
+    TaskName mockTaskName = mock(TaskName.class);
     TaskContextImpl mockContext = mock(TaskContextImpl.class);
+    when(mockContext.getTaskName()).thenReturn(mockTaskName);
     when(mockContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap());
-    StreamGraphImpl streamGraph = new StreamGraphImpl(mockRunner, mockConfig);
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig);
 
-    MessageStream<Object> inputStream1 = streamGraph.getInputStream("input1");
-    MessageStream<Object> inputStream2 = streamGraph.getInputStream("input2");
+    MessageStream<Object> inputStream1 = graphSpec.getInputStream("input1");
+    MessageStream<Object> inputStream2 = graphSpec.getInputStream("input2");
 
-    List<String> initializedOperators = new ArrayList<>();
-    List<String> closedOperators = new ArrayList<>();
+    Function mapFn = (Function & Serializable) m -> m;
+    inputStream1.map(new TestMapFunction<Object, Object>("1", mapFn))
+        .map(new TestMapFunction<Object, Object>("2", mapFn));
 
-    inputStream1.map(createMapFunction("1", initializedOperators, closedOperators))
-        .map(createMapFunction("2", initializedOperators, closedOperators));
+    inputStream2.map(new TestMapFunction<Object, Object>("3", mapFn))
+        .map(new TestMapFunction<Object, Object>("4", mapFn));
 
-    inputStream2.map(createMapFunction("3", initializedOperators, closedOperators))
-        .map(createMapFunction("4", initializedOperators, closedOperators));
+    OperatorImplGraph opImplGraph = new OperatorImplGraph(graphSpec.getOperatorSpecGraph(), mockConfig, mockContext, SystemClock.instance());
 
-    OperatorImplGraph opImplGraph = new OperatorImplGraph(streamGraph, mockConfig, mockContext, SystemClock.instance());
+    List<String> initializedOperators = BaseTestFunction.getInitListByTaskName(mockTaskName);
 
     // Assert that initialization occurs in topological order.
     assertEquals(initializedOperators.get(0), "1");
@@ -313,35 +466,13 @@ public class TestOperatorImplGraph {
 
     // Assert that finalization occurs in reverse topological order.
     opImplGraph.close();
+    List<String> closedOperators = BaseTestFunction.getCloseListByTaskName(mockTaskName);
     assertEquals(closedOperators.get(0), "4");
     assertEquals(closedOperators.get(1), "3");
     assertEquals(closedOperators.get(2), "2");
     assertEquals(closedOperators.get(3), "1");
   }
 
-  /**
-   * Creates an identity map function that appends to the provided lists when init/close is invoked.
-   */
-  private MapFunction<Object, Object> createMapFunction(String id,
-      List<String> initializedOperators, List<String> finalizedOperators) {
-    return new MapFunction<Object, Object>() {
-      @Override
-      public void init(Config config, TaskContext context) {
-        initializedOperators.add(id);
-      }
-
-      @Override
-      public void close() {
-        finalizedOperators.add(id);
-      }
-
-      @Override
-      public Object apply(Object message) {
-        return message;
-      }
-    };
-  }
-
   @Test
   public void testGetStreamToConsumerTasks() {
     String system = "test-system";
@@ -409,16 +540,16 @@ public class TestOperatorImplGraph {
     when(runner.getStreamSpec("test-app-1-partition_by-p2")).thenReturn(int1);
     when(runner.getStreamSpec("test-app-1-partition_by-p1")).thenReturn(int2);
 
-    StreamGraphImpl streamGraph = new StreamGraphImpl(runner, config);
-    MessageStream messageStream1 = streamGraph.getInputStream("input1").map(m -> m);
-    MessageStream messageStream2 = streamGraph.getInputStream("input2").filter(m -> true);
+    StreamGraphSpec graphSpec = new StreamGraphSpec(runner, config);
+    MessageStream messageStream1 = graphSpec.getInputStream("input1").map(m -> m);
+    MessageStream messageStream2 = graphSpec.getInputStream("input2").filter(m -> true);
     MessageStream messageStream3 =
-        streamGraph.getInputStream("input3")
+        graphSpec.getInputStream("input3")
             .filter(m -> true)
             .partitionBy(m -> "hehe", m -> m, "p1")
             .map(m -> m);
-    OutputStream<Object> outputStream1 = streamGraph.getOutputStream("output1");
-    OutputStream<Object> outputStream2 = streamGraph.getOutputStream("output2");
+    OutputStream<Object> outputStream1 = graphSpec.getOutputStream("output1");
+    OutputStream<Object> outputStream2 = graphSpec.getOutputStream("output2");
 
     messageStream1
         .join(messageStream2, mock(JoinFunction.class),
@@ -430,7 +561,8 @@ public class TestOperatorImplGraph {
             mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(1), "j2")
         .sendTo(outputStream2);
 
-    Multimap<SystemStream, SystemStream> outputToInput = OperatorImplGraph.getIntermediateToInputStreamsMap(streamGraph);
+    Multimap<SystemStream, SystemStream> outputToInput =
+        OperatorImplGraph.getIntermediateToInputStreamsMap(graphSpec.getOperatorSpecGraph());
     Collection<SystemStream> inputs = outputToInput.get(int1.toSystemStream());
     assertEquals(inputs.size(), 2);
     assertTrue(inputs.contains(input1.toSystemStream()));

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java
index a91c1af..873cd3c 100644
--- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java
+++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java
@@ -48,7 +48,7 @@ public class TestStreamOperatorImpl {
     Config mockConfig = mock(Config.class);
     TaskContext mockContext = mock(TaskContext.class);
     StreamOperatorImpl<TestMessageEnvelope, TestOutputMessageEnvelope> opImpl =
-        new StreamOperatorImpl<>(mockOp, mockConfig, mockContext);
+        new StreamOperatorImpl<>(mockOp);
     TestMessageEnvelope inMsg = mock(TestMessageEnvelope.class);
     Collection<TestOutputMessageEnvelope> mockOutputs = mock(Collection.class);
     when(txfmFn.apply(inMsg)).thenReturn(mockOutputs);
@@ -69,7 +69,7 @@ public class TestStreamOperatorImpl {
     TaskContext mockContext = mock(TaskContext.class);
 
     StreamOperatorImpl<TestMessageEnvelope, TestOutputMessageEnvelope> opImpl =
-        new StreamOperatorImpl<>(mockOp, mockConfig, mockContext);
+        new StreamOperatorImpl<>(mockOp);
 
     // ensure that close is not called yet
     verify(txfmFn, times(0)).close();

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java
index 7d0c623..9741fc4 100644
--- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java
+++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestWindowOperator.java
@@ -22,19 +22,20 @@ package org.apache.samza.operators.impl;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
-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.config.MapConfig;
 import org.apache.samza.config.JobConfig;
 import org.apache.samza.container.TaskContextImpl;
 import org.apache.samza.container.TaskName;
 import org.apache.samza.metrics.MetricsRegistryMap;
 import org.apache.samza.operators.KV;
 import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamGraphSpec;
+import org.apache.samza.operators.functions.MapFunction;
 import org.apache.samza.operators.impl.store.TestInMemoryStore;
 import org.apache.samza.operators.impl.store.TimeSeriesKeySerde;
+import org.apache.samza.operators.OperatorSpecGraph;
 import org.apache.samza.operators.triggers.FiringType;
 import org.apache.samza.operators.triggers.Trigger;
 import org.apache.samza.operators.triggers.Triggers;
@@ -54,19 +55,25 @@ import org.apache.samza.task.MessageCollector;
 import org.apache.samza.task.StreamOperatorTask;
 import org.apache.samza.task.TaskCoordinator;
 import org.apache.samza.testUtils.TestClock;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.io.IOException;
 import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
-import java.util.function.Function;
+import java.util.Map;
+import java.util.Collections;
 
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 public class TestWindowOperator {
   private final TaskCoordinator taskCoordinator = mock(TaskCoordinator.class);
@@ -83,26 +90,32 @@ public class TestWindowOperator {
     taskContext = mock(TaskContextImpl.class);
     runner = mock(ApplicationRunner.class);
     Serde storeKeySerde = new TimeSeriesKeySerde(new IntegerSerde());
-    Serde storeValSerde = new IntegerEnvelopeSerde();
+    Serde storeValSerde = KVSerde.of(new IntegerSerde(), new IntegerSerde());
 
     when(taskContext.getSystemStreamPartitions()).thenReturn(ImmutableSet
         .of(new SystemStreamPartition("kafka", "integers", new Partition(0))));
     when(taskContext.getMetricsRegistry()).thenReturn(new MetricsRegistryMap());
-
     when(taskContext.getStore("jobName-jobId-window-w1"))
         .thenReturn(new TestInMemoryStore<>(storeKeySerde, storeValSerde));
     when(runner.getStreamSpec("integers")).thenReturn(new StreamSpec("integers", "integers", "kafka"));
+
+    Map<String, String> mapConfig = new HashMap<>();
+    mapConfig.put("app.runner.class", "org.apache.samza.runtime.LocalApplicationRunner");
+    mapConfig.put("job.default.system", "kafka");
+    mapConfig.put("job.name", "jobName");
+    mapConfig.put("job.id", "jobId");
+    config = new MapConfig(mapConfig);
   }
 
   @Test
   public void testTumblingWindowsDiscardingMode() throws Exception {
 
-    StreamApplication sgb = new KeyedTumblingWindowStreamApplication(AccumulationMode.DISCARDING,
-        Duration.ofSeconds(1), Triggers.repeat(Triggers.count(2)));
+    OperatorSpecGraph sgb = this.getKeyedTumblingWindowStreamGraph(AccumulationMode.DISCARDING,
+        Duration.ofSeconds(1), Triggers.repeat(Triggers.count(2))).getOperatorSpecGraph();
     List<WindowPane<Integer, Collection<IntegerEnvelope>>> windowPanes = new ArrayList<>();
 
     TestClock testClock = new TestClock();
-    StreamOperatorTask task = new StreamOperatorTask(sgb, runner, testClock);
+    StreamOperatorTask task = new StreamOperatorTask(sgb, null, testClock);
     task.init(config, taskContext);
     MessageCollector messageCollector =
         envelope -> windowPanes.add((WindowPane<Integer, Collection<IntegerEnvelope>>) envelope.getMessage());
@@ -130,12 +143,12 @@ public class TestWindowOperator {
   @Test
   public void testNonKeyedTumblingWindowsDiscardingMode() throws Exception {
 
-    StreamApplication sgb = new TumblingWindowStreamApplication(AccumulationMode.DISCARDING,
-        Duration.ofSeconds(1), Triggers.repeat(Triggers.count(1000)));
+    OperatorSpecGraph sgb = this.getTumblingWindowStreamGraph(AccumulationMode.DISCARDING,
+        Duration.ofSeconds(1), Triggers.repeat(Triggers.count(1000))).getOperatorSpecGraph();
     List<WindowPane<Integer, Collection<IntegerEnvelope>>> windowPanes = new ArrayList<>();
 
     TestClock testClock = new TestClock();
-    StreamOperatorTask task = new StreamOperatorTask(sgb, runner, testClock);
+    StreamOperatorTask task = new StreamOperatorTask(sgb, null, testClock);
     task.init(config, taskContext);
 
     MessageCollector messageCollector =
@@ -159,12 +172,12 @@ public class TestWindowOperator {
     when(taskContext.getStore("jobName-jobId-window-w1"))
         .thenReturn(new TestInMemoryStore<>(new TimeSeriesKeySerde(new IntegerSerde()), new IntegerSerde()));
 
-    StreamApplication sgb = new AggregateTumblingWindowStreamApplication(AccumulationMode.DISCARDING,
-        Duration.ofSeconds(1), Triggers.repeat(Triggers.count(2)));
+    OperatorSpecGraph sgb = this.getAggregateTumblingWindowStreamGraph(AccumulationMode.DISCARDING,
+        Duration.ofSeconds(1), Triggers.repeat(Triggers.count(2))).getOperatorSpecGraph();
     List<WindowPane<Integer, Integer>> windowPanes = new ArrayList<>();
 
     TestClock testClock = new TestClock();
-    StreamOperatorTask task = new StreamOperatorTask(sgb, runner, testClock);
+    StreamOperatorTask task = new StreamOperatorTask(sgb, null, testClock);
     task.init(config, taskContext);
     MessageCollector messageCollector = envelope -> windowPanes.add((WindowPane<Integer, Integer>) envelope.getMessage());
     integers.forEach(n -> task.process(new IntegerEnvelope(n), messageCollector, taskCoordinator));
@@ -181,11 +194,11 @@ public class TestWindowOperator {
 
   @Test
   public void testTumblingWindowsAccumulatingMode() throws Exception {
-    StreamApplication sgb = new KeyedTumblingWindowStreamApplication(AccumulationMode.ACCUMULATING,
-        Duration.ofSeconds(1), Triggers.repeat(Triggers.count(2)));
+    OperatorSpecGraph sgb = this.getKeyedTumblingWindowStreamGraph(AccumulationMode.ACCUMULATING,
+        Duration.ofSeconds(1), Triggers.repeat(Triggers.count(2))).getOperatorSpecGraph();
     List<WindowPane<Integer, Collection<IntegerEnvelope>>> windowPanes = new ArrayList<>();
     TestClock testClock = new TestClock();
-    StreamOperatorTask task = new StreamOperatorTask(sgb, runner, testClock);
+    StreamOperatorTask task = new StreamOperatorTask(sgb, null, testClock);
     task.init(config, taskContext);
 
     MessageCollector messageCollector =
@@ -210,10 +223,11 @@ public class TestWindowOperator {
 
   @Test
   public void testSessionWindowsDiscardingMode() throws Exception {
-    StreamApplication sgb = new KeyedSessionWindowStreamApplication(AccumulationMode.DISCARDING, Duration.ofMillis(500));
+    OperatorSpecGraph sgb =
+        this.getKeyedSessionWindowStreamGraph(AccumulationMode.DISCARDING, Duration.ofMillis(500)).getOperatorSpecGraph();
     TestClock testClock = new TestClock();
     List<WindowPane<Integer, Collection<IntegerEnvelope>>> windowPanes = new ArrayList<>();
-    StreamOperatorTask task = new StreamOperatorTask(sgb, runner, testClock);
+    StreamOperatorTask task = new StreamOperatorTask(sgb, null, testClock);
     task.init(config, taskContext);
     MessageCollector messageCollector =
         envelope -> windowPanes.add((WindowPane<Integer, Collection<IntegerEnvelope>>) envelope.getMessage());
@@ -255,10 +269,10 @@ public class TestWindowOperator {
 
   @Test
   public void testSessionWindowsAccumulatingMode() throws Exception {
-    StreamApplication sgb = new KeyedSessionWindowStreamApplication(AccumulationMode.DISCARDING,
-        Duration.ofMillis(500));
+    OperatorSpecGraph sgb = this.getKeyedSessionWindowStreamGraph(AccumulationMode.DISCARDING,
+        Duration.ofMillis(500)).getOperatorSpecGraph();
     TestClock testClock = new TestClock();
-    StreamOperatorTask task = new StreamOperatorTask(sgb, runner, testClock);
+    StreamOperatorTask task = new StreamOperatorTask(sgb, null, testClock);
     List<WindowPane<Integer, Collection<IntegerEnvelope>>> windowPanes = new ArrayList<>();
 
     MessageCollector messageCollector =
@@ -287,10 +301,10 @@ public class TestWindowOperator {
 
   @Test
   public void testCancellationOfOnceTrigger() throws Exception {
-    StreamApplication sgb = new KeyedTumblingWindowStreamApplication(AccumulationMode.ACCUMULATING,
-        Duration.ofSeconds(1), Triggers.count(2));
+    OperatorSpecGraph sgb = this.getKeyedTumblingWindowStreamGraph(AccumulationMode.ACCUMULATING,
+        Duration.ofSeconds(1), Triggers.count(2)).getOperatorSpecGraph();
     TestClock testClock = new TestClock();
-    StreamOperatorTask task = new StreamOperatorTask(sgb, runner, testClock);
+    StreamOperatorTask task = new StreamOperatorTask(sgb, null, testClock);
     task.init(config, taskContext);
 
     List<WindowPane<Integer, Collection<IntegerEnvelope>>> windowPanes = new ArrayList<>();
@@ -331,10 +345,10 @@ public class TestWindowOperator {
 
   @Test
   public void testCancellationOfAnyTrigger() throws Exception {
-    StreamApplication sgb = new KeyedTumblingWindowStreamApplication(AccumulationMode.ACCUMULATING, Duration.ofSeconds(1),
-        Triggers.any(Triggers.count(2), Triggers.timeSinceFirstMessage(Duration.ofMillis(500))));
+    OperatorSpecGraph sgb = this.getKeyedTumblingWindowStreamGraph(AccumulationMode.ACCUMULATING, Duration.ofSeconds(1),
+        Triggers.any(Triggers.count(2), Triggers.timeSinceFirstMessage(Duration.ofMillis(500)))).getOperatorSpecGraph();
     TestClock testClock = new TestClock();
-    StreamOperatorTask task = new StreamOperatorTask(sgb, runner, testClock);
+    StreamOperatorTask task = new StreamOperatorTask(sgb, null, testClock);
     task.init(config, taskContext);
 
     List<WindowPane<Integer, Collection<IntegerEnvelope>>> windowPanes = new ArrayList<>();
@@ -389,15 +403,15 @@ public class TestWindowOperator {
   @Test
   public void testCancelationOfRepeatingNestedTriggers() throws Exception {
 
-    StreamApplication sgb = new KeyedTumblingWindowStreamApplication(AccumulationMode.ACCUMULATING, Duration.ofSeconds(1),
-        Triggers.repeat(Triggers.any(Triggers.count(2), Triggers.timeSinceFirstMessage(Duration.ofMillis(500)))));
+    OperatorSpecGraph sgb = this.getKeyedTumblingWindowStreamGraph(AccumulationMode.ACCUMULATING, Duration.ofSeconds(1),
+        Triggers.repeat(Triggers.any(Triggers.count(2), Triggers.timeSinceFirstMessage(Duration.ofMillis(500))))).getOperatorSpecGraph();
     List<WindowPane<Integer, Collection<IntegerEnvelope>>> windowPanes = new ArrayList<>();
 
     MessageCollector messageCollector =
         envelope -> windowPanes.add((WindowPane<Integer, Collection<IntegerEnvelope>>) envelope.getMessage());
 
     TestClock testClock = new TestClock();
-    StreamOperatorTask task = new StreamOperatorTask(sgb, runner, testClock);
+    StreamOperatorTask task = new StreamOperatorTask(sgb, null, testClock);
     task.init(config, taskContext);
 
     task.process(new IntegerEnvelope(1), messageCollector, taskCoordinator);
@@ -434,12 +448,12 @@ public class TestWindowOperator {
     when(taskContext.fetchObject(EndOfStreamStates.class.getName())).thenReturn(endOfStreamStates);
     when(taskContext.fetchObject(WatermarkStates.class.getName())).thenReturn(mock(WatermarkStates.class));
 
-    StreamApplication sgb = new TumblingWindowStreamApplication(AccumulationMode.DISCARDING,
-        Duration.ofSeconds(1), Triggers.repeat(Triggers.count(2)));
+    OperatorSpecGraph sgb = this.getTumblingWindowStreamGraph(AccumulationMode.DISCARDING,
+        Duration.ofSeconds(1), Triggers.repeat(Triggers.count(2))).getOperatorSpecGraph();
     List<WindowPane<Integer, Collection<IntegerEnvelope>>> windowPanes = new ArrayList<>();
 
     TestClock testClock = new TestClock();
-    StreamOperatorTask task = new StreamOperatorTask(sgb, runner, testClock);
+    StreamOperatorTask task = new StreamOperatorTask(sgb, null, testClock);
     task.init(config, taskContext);
 
     MessageCollector messageCollector =
@@ -475,10 +489,11 @@ public class TestWindowOperator {
     when(taskContext.fetchObject(EndOfStreamStates.class.getName())).thenReturn(endOfStreamStates);
     when(taskContext.fetchObject(WatermarkStates.class.getName())).thenReturn(mock(WatermarkStates.class));
 
-    StreamApplication sgb = new KeyedSessionWindowStreamApplication(AccumulationMode.DISCARDING, Duration.ofMillis(500));
+    OperatorSpecGraph sgb =
+        this.getKeyedSessionWindowStreamGraph(AccumulationMode.DISCARDING, Duration.ofMillis(500)).getOperatorSpecGraph();
     TestClock testClock = new TestClock();
     List<WindowPane<Integer, Collection<IntegerEnvelope>>> windowPanes = new ArrayList<>();
-    StreamOperatorTask task = new StreamOperatorTask(sgb, runner, testClock);
+    StreamOperatorTask task = new StreamOperatorTask(sgb, null, testClock);
     task.init(config, taskContext);
 
     MessageCollector messageCollector =
@@ -511,10 +526,11 @@ public class TestWindowOperator {
     when(taskContext.fetchObject(EndOfStreamStates.class.getName())).thenReturn(endOfStreamStates);
     when(taskContext.fetchObject(WatermarkStates.class.getName())).thenReturn(mock(WatermarkStates.class));
 
-    StreamApplication sgb = new KeyedSessionWindowStreamApplication(AccumulationMode.DISCARDING, Duration.ofMillis(500));
+    OperatorSpecGraph sgb =
+        this.getKeyedSessionWindowStreamGraph(AccumulationMode.DISCARDING, Duration.ofMillis(500)).getOperatorSpecGraph();
     TestClock testClock = new TestClock();
     List<WindowPane<Integer, Collection<IntegerEnvelope>>> windowPanes = new ArrayList<>();
-    StreamOperatorTask task = new StreamOperatorTask(sgb, runner, testClock);
+    StreamOperatorTask task = new StreamOperatorTask(sgb, null, testClock);
     task.init(config, taskContext);
 
     MessageCollector messageCollector =
@@ -534,144 +550,83 @@ public class TestWindowOperator {
     verify(taskCoordinator, times(1)).shutdown(TaskCoordinator.RequestScope.CURRENT_TASK);
   }
 
-  private class KeyedTumblingWindowStreamApplication implements StreamApplication {
-
-    private final AccumulationMode mode;
-    private final Duration duration;
-    private final Trigger<IntegerEnvelope> earlyTrigger;
-    private final SystemStream outputSystemStream = new SystemStream("outputSystem", "outputStream");
+  private StreamGraphSpec getKeyedTumblingWindowStreamGraph(AccumulationMode mode,
+      Duration duration, Trigger<KV<Integer, Integer>> earlyTrigger) throws IOException {
+    StreamGraphSpec graph = new StreamGraphSpec(runner, config);
 
-    KeyedTumblingWindowStreamApplication(AccumulationMode mode,
-        Duration timeDuration, Trigger<IntegerEnvelope> earlyTrigger) {
-      this.mode = mode;
-      this.duration = timeDuration;
-      this.earlyTrigger = earlyTrigger;
-    }
+    KVSerde<Integer, Integer> kvSerde = KVSerde.of(new IntegerSerde(), new IntegerSerde());
+    graph.getInputStream("integers", kvSerde)
+        .window(Windows.keyedTumblingWindow(KV::getKey, duration, new IntegerSerde(), kvSerde)
+            .setEarlyTrigger(earlyTrigger).setAccumulationMode(mode), "w1")
+        .sink((message, messageCollector, taskCoordinator) -> {
+            SystemStream outputSystemStream = new SystemStream("outputSystem", "outputStream");
+            messageCollector.send(new OutgoingMessageEnvelope(outputSystemStream, message));
+          });
 
-    @Override
-    public void init(StreamGraph graph, Config config) {
-      MessageStream<IntegerEnvelope> inStream =
-          graph.getInputStream("integers", KVSerde.of(new IntegerSerde(), new IntegerSerde()))
-              .map(kv -> new IntegerEnvelope(kv.getKey()));
-      Function<IntegerEnvelope, Integer> keyFn = m -> (Integer) m.getKey();
-      inStream
-          .map(m -> m)
-          .window(Windows.keyedTumblingWindow(keyFn, duration, new IntegerSerde(), new IntegerEnvelopeSerde())
-              .setEarlyTrigger(earlyTrigger)
-              .setAccumulationMode(mode), "w1")
-          .sink((message, messageCollector, taskCoordinator) -> {
-              messageCollector.send(new OutgoingMessageEnvelope(outputSystemStream, message));
-            });
-    }
+    return graph;
   }
 
-  private class TumblingWindowStreamApplication implements StreamApplication {
-
-    private final AccumulationMode mode;
-    private final Duration duration;
-    private final Trigger<IntegerEnvelope> earlyTrigger;
-    private final SystemStream outputSystemStream = new SystemStream("outputSystem", "outputStream");
+  private StreamGraphSpec getTumblingWindowStreamGraph(AccumulationMode mode,
+      Duration duration, Trigger<KV<Integer, Integer>> earlyTrigger) throws IOException {
+    StreamGraphSpec graph = new StreamGraphSpec(runner, config);
 
-    TumblingWindowStreamApplication(AccumulationMode mode,
-                                         Duration timeDuration, Trigger<IntegerEnvelope> earlyTrigger) {
-      this.mode = mode;
-      this.duration = timeDuration;
-      this.earlyTrigger = earlyTrigger;
-    }
-
-    @Override
-    public void init(StreamGraph graph, Config config) {
-      MessageStream<IntegerEnvelope> inStream =
-          graph.getInputStream("integers", KVSerde.of(new IntegerSerde(), new IntegerSerde()))
-              .map(kv -> new IntegerEnvelope(kv.getKey()));
-      Function<IntegerEnvelope, Integer> keyFn = m -> (Integer) m.getKey();
-      inStream
-          .map(m -> m)
-          .window(Windows.tumblingWindow(duration, new IntegerEnvelopeSerde())
-              .setEarlyTrigger(earlyTrigger)
-              .setAccumulationMode(mode), "w1")
-          .sink((message, messageCollector, taskCoordinator) -> {
-              messageCollector.send(new OutgoingMessageEnvelope(outputSystemStream, message));
-            });
-    }
+    KVSerde<Integer, Integer> kvSerde = KVSerde.of(new IntegerSerde(), new IntegerSerde());
+    graph.getInputStream("integers", kvSerde)
+        .window(Windows.tumblingWindow(duration, kvSerde).setEarlyTrigger(earlyTrigger)
+            .setAccumulationMode(mode), "w1")
+        .sink((message, messageCollector, taskCoordinator) -> {
+            SystemStream outputSystemStream = new SystemStream("outputSystem", "outputStream");
+            messageCollector.send(new OutgoingMessageEnvelope(outputSystemStream, message));
+          });
+    return graph;
   }
 
-  private class AggregateTumblingWindowStreamApplication implements StreamApplication {
+  private StreamGraphSpec getKeyedSessionWindowStreamGraph(AccumulationMode mode, Duration duration) throws IOException {
+    StreamGraphSpec graph = new StreamGraphSpec(runner, config);
 
-    private final AccumulationMode mode;
-    private final Duration duration;
-    private final Trigger<IntegerEnvelope> earlyTrigger;
-    private final SystemStream outputSystemStream = new SystemStream("outputSystem", "outputStream");
-
-    AggregateTumblingWindowStreamApplication(AccumulationMode mode, Duration timeDuration,
-        Trigger<IntegerEnvelope> earlyTrigger) {
-      this.mode = mode;
-      this.duration = timeDuration;
-      this.earlyTrigger = earlyTrigger;
-    }
-
-    @Override
-    public void init(StreamGraph graph, Config config) {
-      MessageStream<KV<Integer, Integer>> integers = graph.getInputStream("integers",
-          KVSerde.of(new IntegerSerde(), new IntegerSerde()));
-
-      integers
-        .map(kv -> new IntegerEnvelope(kv.getKey()))
-        .window(Windows.<IntegerEnvelope, Integer>tumblingWindow(this.duration, () -> 0, (m, c) -> c + 1, new IntegerSerde())
-            .setEarlyTrigger(earlyTrigger)
+    KVSerde<Integer, Integer> kvSerde = KVSerde.of(new IntegerSerde(), new IntegerSerde());
+    graph.getInputStream("integers", kvSerde)
+        .window(Windows.keyedSessionWindow(KV::getKey, duration, new IntegerSerde(), kvSerde)
             .setAccumulationMode(mode), "w1")
         .sink((message, messageCollector, taskCoordinator) -> {
+            SystemStream outputSystemStream = new SystemStream("outputSystem", "outputStream");
             messageCollector.send(new OutgoingMessageEnvelope(outputSystemStream, message));
           });
-    }
+    return graph;
   }
 
-  private class KeyedSessionWindowStreamApplication implements StreamApplication {
+  private StreamGraphSpec getAggregateTumblingWindowStreamGraph(AccumulationMode mode, Duration timeDuration,
+        Trigger<IntegerEnvelope> earlyTrigger) throws IOException {
+    StreamGraphSpec graph = new StreamGraphSpec(runner, config);
 
-    private final AccumulationMode mode;
-    private final Duration duration;
-    private final SystemStream outputSystemStream = new SystemStream("outputSystem", "outputStream");
+    MessageStream<KV<Integer, Integer>> integers = graph.getInputStream("integers",
+        KVSerde.of(new IntegerSerde(), new IntegerSerde()));
 
-    KeyedSessionWindowStreamApplication(AccumulationMode mode, Duration duration) {
-      this.mode = mode;
-      this.duration = duration;
-    }
-
-    @Override
-    public void init(StreamGraph graph, Config config) {
-      MessageStream<IntegerEnvelope> inStream =
-          graph.getInputStream("integers", KVSerde.of(new IntegerSerde(), new IntegerSerde()))
-              .map(kv -> new IntegerEnvelope(kv.getKey()));
-      Function<IntegerEnvelope, Integer> keyFn = m -> (Integer) m.getKey();
-
-      inStream
-          .map(m -> m)
-          .window(Windows.keyedSessionWindow(keyFn, duration, new IntegerSerde(), new IntegerEnvelopeSerde())
-              .setAccumulationMode(mode), "w1")
-          .sink((message, messageCollector, taskCoordinator) -> {
-              messageCollector.send(new OutgoingMessageEnvelope(outputSystemStream, message));
-            });
-    }
+    integers
+        .map(new KVMapFunction())
+        .window(Windows.<IntegerEnvelope, Integer>tumblingWindow(timeDuration, () -> 0, (m, c) -> c + 1, new IntegerSerde())
+            .setEarlyTrigger(earlyTrigger)
+            .setAccumulationMode(mode), "w1")
+        .sink((message, messageCollector, taskCoordinator) -> {
+            SystemStream outputSystemStream = new SystemStream("outputSystem", "outputStream");
+            messageCollector.send(new OutgoingMessageEnvelope(outputSystemStream, message));
+          });
+    return graph;
   }
 
-  private class IntegerEnvelope extends IncomingMessageEnvelope  {
+  private static class IntegerEnvelope extends IncomingMessageEnvelope {
 
     IntegerEnvelope(Integer key) {
-      super(new SystemStreamPartition("kafka", "integers", new Partition(0)), "1", key, key);
+      super(new SystemStreamPartition("kafka", "integers", new Partition(0)), null, key, key);
     }
   }
 
-  private class IntegerEnvelopeSerde implements Serde<IntegerEnvelope> {
-    private final IntegerSerde intSerde = new IntegerSerde();
+  private static class KVMapFunction implements MapFunction<KV<Integer, Integer>, IntegerEnvelope> {
 
     @Override
-    public byte[] toBytes(IntegerEnvelope object) {
-      return intSerde.toBytes((Integer) object.getKey());
-    }
-
-    @Override
-    public IntegerEnvelope fromBytes(byte[] bytes) {
-      return new IntegerEnvelope(intSerde.fromBytes(bytes));
+    public IntegerEnvelope apply(KV<Integer, Integer> message) {
+      return new IntegerEnvelope(message.getKey());
     }
   }
+
 }


[02/10] samza git commit: SAMZA-1659: Serializable OperatorSpec

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/main/java/org/apache/samza/example/BroadcastExample.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/example/BroadcastExample.java b/samza-test/src/main/java/org/apache/samza/example/BroadcastExample.java
new file mode 100644
index 0000000..9ca4f35
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/example/BroadcastExample.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.example;
+
+import org.apache.samza.application.StreamApplication;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.KV;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.runtime.LocalApplicationRunner;
+import org.apache.samza.serializers.JsonSerdeV2;
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.util.CommandLine;
+
+
+/**
+ * Example implementation of a task that splits its input into multiple output streams.
+ */
+public class BroadcastExample implements StreamApplication {
+
+  // local execution mode
+  public static void main(String[] args) throws Exception {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+
+    StreamApplication app = new BroadcastExample();
+    LocalApplicationRunner runner = new LocalApplicationRunner(config);
+
+    runner.run(app);
+    runner.waitForFinish();
+  }
+
+  @Override
+  public void init(StreamGraph graph, Config config) {
+    KVSerde<String, PageViewEvent> pgeMsgSerde = KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class));
+    MessageStream<KV<String, PageViewEvent>> inputStream = graph.getInputStream("pageViewEventStream", pgeMsgSerde);
+
+    inputStream.filter(m -> m.key.equals("key1")).sendTo(graph.getOutputStream("outStream1", pgeMsgSerde));
+    inputStream.filter(m -> m.key.equals("key2")).sendTo(graph.getOutputStream("outStream2", pgeMsgSerde));
+    inputStream.filter(m -> m.key.equals("key3")).sendTo(graph.getOutputStream("outStream3", pgeMsgSerde));
+  }
+
+  class PageViewEvent {
+    String key;
+    long timestamp;
+
+    public PageViewEvent(String key, long timestamp) {
+      this.key = key;
+      this.timestamp = timestamp;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/main/java/org/apache/samza/example/KeyValueStoreExample.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/example/KeyValueStoreExample.java b/samza-test/src/main/java/org/apache/samza/example/KeyValueStoreExample.java
new file mode 100644
index 0000000..9edaabe
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/example/KeyValueStoreExample.java
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.example;
+
+
+import org.apache.samza.application.StreamApplication;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.KV;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.OutputStream;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.runtime.LocalApplicationRunner;
+import org.apache.samza.serializers.JsonSerdeV2;
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.storage.kv.KeyValueStore;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.util.CommandLine;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+
+/**
+ * Example code using {@link KeyValueStore} to implement event-time window
+ */
+public class KeyValueStoreExample implements StreamApplication {
+
+  // local execution mode
+  public static void main(String[] args) throws Exception {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+    KeyValueStoreExample app = new KeyValueStoreExample();
+    LocalApplicationRunner runner = new LocalApplicationRunner(config);
+
+    runner.run(app);
+    runner.waitForFinish();
+  }
+
+  @Override
+  public void init(StreamGraph graph, Config config) {
+    MessageStream<PageViewEvent> pageViewEvents =
+        graph.getInputStream("pageViewEventStream", new JsonSerdeV2<>(PageViewEvent.class));
+    OutputStream<KV<String, StatsOutput>> pageViewEventPerMember =
+        graph.getOutputStream("pageViewEventPerMember",
+            KVSerde.of(new StringSerde(), new JsonSerdeV2<>(StatsOutput.class)));
+
+    pageViewEvents
+        .partitionBy(pve -> pve.memberId, pve -> pve,
+            KVSerde.of(new StringSerde(), new JsonSerdeV2<>(PageViewEvent.class)), "partitionBy")
+        .map(KV::getValue)
+        .flatMap(new MyStatsCounter())
+        .map(stats -> KV.of(stats.memberId, stats))
+        .sendTo(pageViewEventPerMember);
+  }
+
+  static class MyStatsCounter implements FlatMapFunction<PageViewEvent, StatsOutput> {
+    private final int timeoutMs = 10 * 60 * 1000;
+
+    KeyValueStore<String, StatsWindowState> statsStore;
+
+    class StatsWindowState {
+      int lastCount = 0;
+      long timeAtLastOutput = 0;
+      int newCount = 0;
+    }
+
+    @Override
+    public Collection<StatsOutput> apply(PageViewEvent message) {
+      List<StatsOutput> outputStats = new ArrayList<>();
+      long wndTimestamp = (long) Math.floor(TimeUnit.MILLISECONDS.toMinutes(message.timestamp) / 5) * 5;
+      String wndKey = String.format("%s-%d", message.memberId, wndTimestamp);
+      StatsWindowState curState = this.statsStore.get(wndKey);
+      if (curState == null) {
+        curState = new StatsWindowState();
+      }
+      curState.newCount++;
+      long curTimeMs = System.currentTimeMillis();
+      if (curState.newCount > 0 && curState.timeAtLastOutput + timeoutMs < curTimeMs) {
+        curState.timeAtLastOutput = curTimeMs;
+        curState.lastCount += curState.newCount;
+        curState.newCount = 0;
+        outputStats.add(new StatsOutput(message.memberId, wndTimestamp, curState.lastCount));
+      }
+      // update counter w/o generating output
+      this.statsStore.put(wndKey, curState);
+      return outputStats;
+    }
+
+    @Override
+    public void init(Config config, TaskContext context) {
+      this.statsStore = (KeyValueStore<String, StatsWindowState>) context.getStore("my-stats-wnd-store");
+    }
+  }
+
+  class PageViewEvent {
+    String pageId;
+    String memberId;
+    long timestamp;
+
+    PageViewEvent(String pageId, String memberId, long timestamp) {
+      this.pageId = pageId;
+      this.memberId = memberId;
+      this.timestamp = timestamp;
+    }
+  }
+
+  static class StatsOutput {
+    private String memberId;
+    private long timestamp;
+    private Integer count;
+
+    StatsOutput(String key, long timestamp, Integer count) {
+      this.memberId = key;
+      this.timestamp = timestamp;
+      this.count = count;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/main/java/org/apache/samza/example/MergeExample.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/example/MergeExample.java b/samza-test/src/main/java/org/apache/samza/example/MergeExample.java
new file mode 100644
index 0000000..ff983a4
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/example/MergeExample.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.example;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.samza.application.StreamApplication;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.runtime.LocalApplicationRunner;
+import org.apache.samza.serializers.JsonSerdeV2;
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.util.CommandLine;
+
+public class MergeExample implements StreamApplication {
+
+  // local execution mode
+  public static void main(String[] args) throws Exception {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+    MergeExample app = new MergeExample();
+    LocalApplicationRunner runner = new LocalApplicationRunner(config);
+
+    runner.run(app);
+    runner.waitForFinish();
+  }
+
+  @Override
+  public void init(StreamGraph graph, Config config) {
+
+    KVSerde<String, PageViewEvent>
+        pgeMsgSerde = KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class));
+
+    MessageStream.mergeAll(ImmutableList.of(graph.getInputStream("viewStream1", pgeMsgSerde),
+        graph.getInputStream("viewStream2", pgeMsgSerde), graph.getInputStream("viewStream3", pgeMsgSerde)))
+        .sendTo(graph.getOutputStream("mergedStream", pgeMsgSerde));
+
+  }
+
+  class PageViewEvent {
+    String pageId;
+    long viewTimestamp;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/main/java/org/apache/samza/example/OrderShipmentJoinExample.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/example/OrderShipmentJoinExample.java b/samza-test/src/main/java/org/apache/samza/example/OrderShipmentJoinExample.java
new file mode 100644
index 0000000..1c0bc25
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/example/OrderShipmentJoinExample.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.example;
+
+import org.apache.samza.application.StreamApplication;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.KV;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.OutputStream;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.functions.JoinFunction;
+import org.apache.samza.runtime.LocalApplicationRunner;
+import org.apache.samza.serializers.JsonSerdeV2;
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.util.CommandLine;
+
+import java.time.Duration;
+
+
+/**
+ * Simple 2-way stream-to-stream join example
+ */
+public class OrderShipmentJoinExample implements StreamApplication {
+
+  // local execution mode
+  public static void main(String[] args) throws Exception {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+    OrderShipmentJoinExample app = new OrderShipmentJoinExample();
+    LocalApplicationRunner runner = new LocalApplicationRunner(config);
+
+    runner.run(app);
+    runner.waitForFinish();
+  }
+
+  @Override
+  public void init(StreamGraph graph, Config config) {
+
+    MessageStream<OrderRecord> orders =
+        graph.getInputStream("orders", new JsonSerdeV2<>(OrderRecord.class));
+    MessageStream<ShipmentRecord> shipments =
+        graph.getInputStream("shipments", new JsonSerdeV2<>(ShipmentRecord.class));
+    OutputStream<KV<String, FulfilledOrderRecord>> fulfilledOrders =
+        graph.getOutputStream("fulfilledOrders",
+            KVSerde.of(new StringSerde(), new JsonSerdeV2<>(FulfilledOrderRecord.class)));
+
+    orders
+        .join(shipments, new MyJoinFunction(),
+            new StringSerde(), new JsonSerdeV2<>(OrderRecord.class), new JsonSerdeV2<>(ShipmentRecord.class),
+            Duration.ofMinutes(1), "join")
+        .map(fulFilledOrder -> KV.of(fulFilledOrder.orderId, fulFilledOrder))
+        .sendTo(fulfilledOrders);
+
+  }
+
+  static class MyJoinFunction implements JoinFunction<String, OrderRecord, ShipmentRecord, FulfilledOrderRecord> {
+    @Override
+    public FulfilledOrderRecord apply(OrderRecord message, ShipmentRecord otherMessage) {
+      return new FulfilledOrderRecord(message.orderId, message.orderTimeMs, otherMessage.shipTimeMs);
+    }
+
+    @Override
+    public String getFirstKey(OrderRecord message) {
+      return message.orderId;
+    }
+
+    @Override
+    public String getSecondKey(ShipmentRecord message) {
+      return message.orderId;
+    }
+  }
+
+  class OrderRecord {
+    String orderId;
+    long orderTimeMs;
+
+    OrderRecord(String orderId, long timeMs) {
+      this.orderId = orderId;
+      this.orderTimeMs = timeMs;
+    }
+  }
+
+  class ShipmentRecord {
+    String orderId;
+    long shipTimeMs;
+
+    ShipmentRecord(String orderId, long timeMs) {
+      this.orderId = orderId;
+      this.shipTimeMs = timeMs;
+    }
+  }
+
+  static class FulfilledOrderRecord {
+    String orderId;
+    long orderTimeMs;
+    long shipTimeMs;
+
+    FulfilledOrderRecord(String orderId, long orderTimeMs, long shipTimeMs) {
+      this.orderId = orderId;
+      this.orderTimeMs = orderTimeMs;
+      this.shipTimeMs = shipTimeMs;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/main/java/org/apache/samza/example/PageViewCounterExample.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/example/PageViewCounterExample.java b/samza-test/src/main/java/org/apache/samza/example/PageViewCounterExample.java
new file mode 100644
index 0000000..2581506
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/example/PageViewCounterExample.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.example;
+
+import java.time.Duration;
+import org.apache.samza.application.StreamApplication;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.KV;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.OutputStream;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.functions.FoldLeftFunction;
+import org.apache.samza.operators.functions.SupplierFunction;
+import org.apache.samza.operators.triggers.Triggers;
+import org.apache.samza.operators.windows.AccumulationMode;
+import org.apache.samza.operators.windows.WindowPane;
+import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.runtime.LocalApplicationRunner;
+import org.apache.samza.serializers.JsonSerdeV2;
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.util.CommandLine;
+
+
+/**
+ * Example code to implement window-based counter
+ */
+public class PageViewCounterExample implements StreamApplication {
+
+  // local execution mode
+  public static void main(String[] args) {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+    PageViewCounterExample app = new PageViewCounterExample();
+    LocalApplicationRunner runner = new LocalApplicationRunner(config);
+
+    runner.run(app);
+    runner.waitForFinish();
+  }
+
+  @Override
+  public void init(StreamGraph graph, Config config) {
+
+    MessageStream<PageViewEvent> pageViewEvents = null;
+    pageViewEvents = graph.getInputStream("pageViewEventStream", new JsonSerdeV2<>(PageViewEvent.class));
+    OutputStream<KV<String, PageViewCount>> pageViewEventPerMemberStream =
+        graph.getOutputStream("pageViewEventPerMemberStream",
+            KVSerde.of(new StringSerde(), new JsonSerdeV2<>(PageViewCount.class)));
+
+    SupplierFunction<Integer> initialValue = () -> 0;
+    FoldLeftFunction<PageViewEvent, Integer> foldLeftFn = (m, c) -> c + 1;
+    pageViewEvents
+        .window(Windows.keyedTumblingWindow(m -> m.memberId, Duration.ofSeconds(10), initialValue, foldLeftFn, null, null)
+            .setEarlyTrigger(Triggers.repeat(Triggers.count(5)))
+            .setAccumulationMode(AccumulationMode.DISCARDING), "tumblingWindow")
+        .map(windowPane -> KV.of(windowPane.getKey().getKey(), new PageViewCount(windowPane)))
+        .sendTo(pageViewEventPerMemberStream);
+
+  }
+
+  class PageViewEvent {
+    String pageId;
+    String memberId;
+    long timestamp;
+
+    PageViewEvent(String pageId, String memberId, long timestamp) {
+      this.pageId = pageId;
+      this.memberId = memberId;
+      this.timestamp = timestamp;
+    }
+  }
+
+  static class PageViewCount {
+    String memberId;
+    long timestamp;
+    int count;
+
+    PageViewCount(WindowPane<String, Integer> m) {
+      this.memberId = m.getKey().getKey();
+      this.timestamp = Long.valueOf(m.getKey().getPaneId());
+      this.count = m.getMessage();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/main/java/org/apache/samza/example/RepartitionExample.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/example/RepartitionExample.java b/samza-test/src/main/java/org/apache/samza/example/RepartitionExample.java
new file mode 100644
index 0000000..7f28346
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/example/RepartitionExample.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.example;
+
+import org.apache.samza.application.StreamApplication;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.KV;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.OutputStream;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.windows.WindowPane;
+import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.runtime.LocalApplicationRunner;
+import org.apache.samza.serializers.JsonSerdeV2;
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.util.CommandLine;
+
+import java.time.Duration;
+
+
+/**
+ * Example {@link StreamApplication} code to test the API methods with re-partition operator
+ */
+public class RepartitionExample implements StreamApplication {
+
+  // local execution mode
+  public static void main(String[] args) throws Exception {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+    RepartitionExample app = new RepartitionExample();
+    LocalApplicationRunner runner = new LocalApplicationRunner(config);
+
+    runner.run(app);
+    runner.waitForFinish();
+  }
+
+  @Override
+  public void init(StreamGraph graph, Config config) {
+
+    MessageStream<PageViewEvent> pageViewEvents =
+        graph.getInputStream("pageViewEvent", new JsonSerdeV2<>(PageViewEvent.class));
+    OutputStream<KV<String, MyStreamOutput>> pageViewEventPerMember =
+        graph.getOutputStream("pageViewEventPerMember",
+            KVSerde.of(new StringSerde(), new JsonSerdeV2<>(MyStreamOutput.class)));
+
+    pageViewEvents
+        .partitionBy(pve -> pve.memberId, pve -> pve,
+            KVSerde.of(new StringSerde(), new JsonSerdeV2<>(PageViewEvent.class)), "partitionBy")
+        .window(Windows.keyedTumblingWindow(KV::getKey, Duration.ofMinutes(5), () -> 0, (m, c) -> c + 1, null, null),
+            "window")
+        .map(windowPane -> KV.of(windowPane.getKey().getKey(), new MyStreamOutput(windowPane)))
+        .sendTo(pageViewEventPerMember);
+
+  }
+
+  class PageViewEvent {
+    String pageId;
+    String memberId;
+    long timestamp;
+
+    PageViewEvent(String pageId, String memberId, long timestamp) {
+      this.pageId = pageId;
+      this.memberId = memberId;
+      this.timestamp = timestamp;
+    }
+  }
+
+  static class MyStreamOutput {
+    String memberId;
+    long timestamp;
+    int count;
+
+    MyStreamOutput(WindowPane<String, Integer> m) {
+      this.memberId = m.getKey().getKey();
+      this.timestamp = Long.valueOf(m.getKey().getPaneId());
+      this.count = m.getMessage();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/main/java/org/apache/samza/example/WindowExample.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/example/WindowExample.java b/samza-test/src/main/java/org/apache/samza/example/WindowExample.java
new file mode 100644
index 0000000..4950695
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/example/WindowExample.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.example;
+
+import org.apache.samza.application.StreamApplication;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.OutputStream;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.functions.FoldLeftFunction;
+import org.apache.samza.operators.functions.SupplierFunction;
+import org.apache.samza.operators.windows.WindowPane;
+import org.apache.samza.operators.triggers.Triggers;
+import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.runtime.LocalApplicationRunner;
+import org.apache.samza.serializers.IntegerSerde;
+import org.apache.samza.serializers.JsonSerdeV2;
+import org.apache.samza.util.CommandLine;
+
+import java.time.Duration;
+
+
+/**
+ * Example implementation of a simple user-defined task w/ a window operator.
+ *
+ */
+public class WindowExample implements StreamApplication {
+
+  // local execution mode
+  public static void main(String[] args) throws Exception {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+    WindowExample app = new WindowExample();
+    LocalApplicationRunner runner = new LocalApplicationRunner(config);
+
+    runner.run(app);
+    runner.waitForFinish();
+  }
+
+  @Override
+  public void init(StreamGraph graph, Config config) {
+
+    SupplierFunction<Integer> initialValue = () -> 0;
+    FoldLeftFunction<PageViewEvent, Integer> counter = (m, c) -> c == null ? 1 : c + 1;
+    MessageStream<PageViewEvent> inputStream = graph.getInputStream("inputStream", new JsonSerdeV2<PageViewEvent>());
+    OutputStream<Integer> outputStream = graph.getOutputStream("outputStream", new IntegerSerde());
+
+    // create a tumbling window that outputs the number of message collected every 10 minutes.
+    // also emit early results if either the number of messages collected reaches 30000, or if no new messages arrive
+    // for 1 minute.
+    inputStream
+        .window(Windows.tumblingWindow(Duration.ofMinutes(10), initialValue, counter, new IntegerSerde())
+            .setLateTrigger(Triggers.any(Triggers.count(30000),
+                Triggers.timeSinceLastMessage(Duration.ofMinutes(1)))), "window")
+        .map(WindowPane::getMessage)
+        .sendTo(outputStream);
+
+  }
+
+  class PageViewEvent {
+    String key;
+    long timestamp;
+
+    public PageViewEvent(String key, long timestamp) {
+      this.key = key;
+      this.timestamp = timestamp;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/main/java/org/apache/samza/test/framework/StreamAssert.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/StreamAssert.java b/samza-test/src/main/java/org/apache/samza/test/framework/StreamAssert.java
index de0d962..a1ac299 100644
--- a/samza-test/src/main/java/org/apache/samza/test/framework/StreamAssert.java
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/StreamAssert.java
@@ -20,6 +20,8 @@
 package org.apache.samza.test.framework;
 
 import com.google.common.collect.Iterables;
+import java.io.IOException;
+import java.io.ObjectInputStream;
 import org.apache.samza.config.Config;
 import org.apache.samza.operators.MessageStream;
 import org.apache.samza.operators.functions.SinkFunction;
@@ -155,6 +157,18 @@ public class StreamAssert<M> {
       }
     }
 
+    private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+      in.defaultReadObject();
+      timer = new Timer();
+      actual = Collections.synchronizedList(new ArrayList<>());
+      timerTask = new TimerTask() {
+        @Override
+        public void run() {
+          check();
+        }
+      };
+    }
+
     private void check() {
       final CountDownLatch latch = LATCHES.get(id);
       try {

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 29c509d..3301af8 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
@@ -40,11 +40,10 @@ 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 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.
@@ -53,6 +52,8 @@ public class EndOfStreamIntegrationTest extends AbstractIntegrationTestHarness {
 
   private static final String[] PAGEKEYS = {"inbox", "home", "search", "pymk", "group", "job"};
 
+  private static List<PageView> received = new ArrayList<>();
+
   @Test
   public void testPipeline() throws  Exception {
     Random random = new Random();
@@ -66,6 +67,7 @@ public class EndOfStreamIntegrationTest extends AbstractIntegrationTestHarness {
 
     int partitionCount = 4;
     Map<String, String> configs = new HashMap<>();
+    configs.put("app.runner.class", "org.apache.samza.runtime.LocalApplicationRunner");
     configs.put("systems.test.samza.factory", ArraySystemFactory.class.getName());
     configs.put("streams.PageView.samza.system", "test");
     configs.put("streams.PageView.source", Base64Serializer.serialize(pageviews));
@@ -89,7 +91,6 @@ public class EndOfStreamIntegrationTest extends AbstractIntegrationTestHarness {
     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.<KV<String, PageView>>getInputStream("PageView")
         .map(Values.create())
@@ -98,6 +99,7 @@ public class EndOfStreamIntegrationTest extends AbstractIntegrationTestHarness {
             received.add(m.getValue());
           });
     };
+
     runner.run(app);
     runner.waitForFinish();
 

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 dda3d24..d4dc4ed 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
@@ -118,6 +118,7 @@ public class WatermarkIntegrationTest extends AbstractIntegrationTestHarness {
   @Test
   public void testWatermark() throws Exception {
     Map<String, String> configs = new HashMap<>();
+    configs.put("app.runner.class", "org.apache.samza.runtime.LocalApplicationRunner");
     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));
@@ -140,7 +141,6 @@ public class WatermarkIntegrationTest extends AbstractIntegrationTestHarness {
     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.<KV<String, PageView>>getInputStream("PageView")
@@ -150,11 +150,14 @@ public class WatermarkIntegrationTest extends AbstractIntegrationTestHarness {
               received.add(m.getValue());
             });
     };
+
+    LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs));
     runner.run(app);
+    // processors are only available when the app is running
     Map<String, StreamOperatorTask> tasks = getTaskOperationGraphs(runner);
 
     runner.waitForFinish();
-
+    // wait for the completion to ensure that all tasks are actually initialized and the OperatorImplGraph is initialized
     StreamOperatorTask task0 = tasks.get("Partition 0");
     OperatorImplGraph graph = TestStreamOperatorTask.getOperatorImplGraph(task0);
     OperatorImpl pb = getOperator(graph, OperatorSpec.OpCode.PARTITION_BY);

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/test/java/org/apache/samza/test/operator/RepartitionJoinWindowApp.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/RepartitionJoinWindowApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/RepartitionJoinWindowApp.java
index 120f902..2171d07 100644
--- a/samza-test/src/test/java/org/apache/samza/test/operator/RepartitionJoinWindowApp.java
+++ b/samza-test/src/test/java/org/apache/samza/test/operator/RepartitionJoinWindowApp.java
@@ -19,17 +19,22 @@
 
 package org.apache.samza.test.operator;
 
+import java.util.ArrayList;
+import java.util.List;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.Config;
 import org.apache.samza.operators.KV;
 import org.apache.samza.operators.MessageStream;
 import org.apache.samza.operators.StreamGraph;
 import org.apache.samza.operators.functions.JoinFunction;
+import org.apache.samza.operators.stream.IntermediateMessageStreamImpl;
 import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.runtime.LocalApplicationRunner;
 import org.apache.samza.serializers.JsonSerdeV2;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.StringSerde;
 import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.StreamSpec;
 import org.apache.samza.system.SystemStream;
 import org.apache.samza.task.TaskCoordinator;
 import org.apache.samza.test.operator.data.AdClick;
@@ -37,6 +42,8 @@ import org.apache.samza.test.operator.data.PageView;
 import org.apache.samza.test.operator.data.UserPageAdClick;
 
 import java.time.Duration;
+import org.apache.samza.util.CommandLine;
+
 
 /**
  * A {@link StreamApplication} that demonstrates a partitionBy, stream-stream join and a windowed count.
@@ -47,6 +54,19 @@ public class RepartitionJoinWindowApp implements StreamApplication {
   public static final String INPUT_TOPIC_NAME_2_PROP = "inputTopicName2";
   public static final String OUTPUT_TOPIC_NAME_PROP = "outputTopicName";
 
+  private final List<StreamSpec> intermediateStreams = new ArrayList<>();
+
+  public static void main(String[] args) throws Exception {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+
+    RepartitionJoinWindowApp application = new RepartitionJoinWindowApp();
+    LocalApplicationRunner runner = new LocalApplicationRunner(config);
+
+    runner.run(application);
+    runner.waitForFinish();
+  }
+
   @Override
   public void init(StreamGraph graph, Config config) {
     String inputTopicName1 = config.get(INPUT_TOPIC_NAME_1_PROP);
@@ -56,25 +76,27 @@ public class RepartitionJoinWindowApp implements StreamApplication {
     MessageStream<PageView> pageViews = graph.getInputStream(inputTopicName1, new JsonSerdeV2<>(PageView.class));
     MessageStream<AdClick> adClicks = graph.getInputStream(inputTopicName2, new JsonSerdeV2<>(AdClick.class));
 
-    MessageStream<PageView> pageViewsRepartitionedByViewId = pageViews
+    MessageStream<KV<String, PageView>> pageViewsRepartitionedByViewId = pageViews
         .partitionBy(PageView::getViewId, pv -> pv,
-            new KVSerde<>(new StringSerde(), new JsonSerdeV2<>(PageView.class)), "pageViewsByViewId")
-        .map(KV::getValue);
+            new KVSerde<>(new StringSerde(), new JsonSerdeV2<>(PageView.class)), "pageViewsByViewId");
+
+    MessageStream<PageView> pageViewsRepartitionedByViewIdValueONly = pageViewsRepartitionedByViewId.map(KV::getValue);
 
-    MessageStream<AdClick> adClicksRepartitionedByViewId = adClicks
+    MessageStream<KV<String, AdClick>> adClicksRepartitionedByViewId = adClicks
         .partitionBy(AdClick::getViewId, ac -> ac,
-            new KVSerde<>(new StringSerde(), new JsonSerdeV2<>(AdClick.class)), "adClicksByViewId")
-        .map(KV::getValue);
+            new KVSerde<>(new StringSerde(), new JsonSerdeV2<>(AdClick.class)), "adClicksByViewId");
+    MessageStream<AdClick> adClicksRepartitionedByViewIdValueOnly = adClicksRepartitionedByViewId.map(KV::getValue);
 
-    MessageStream<UserPageAdClick> userPageAdClicks = pageViewsRepartitionedByViewId
-        .join(adClicksRepartitionedByViewId, new UserPageViewAdClicksJoiner(),
+    MessageStream<UserPageAdClick> userPageAdClicks = pageViewsRepartitionedByViewIdValueONly
+        .join(adClicksRepartitionedByViewIdValueOnly, new UserPageViewAdClicksJoiner(),
             new StringSerde(), new JsonSerdeV2<>(PageView.class), new JsonSerdeV2<>(AdClick.class),
             Duration.ofMinutes(1), "pageViewAdClickJoin");
 
-    userPageAdClicks
+    MessageStream<KV<String, UserPageAdClick>> userPageAdClicksByUserId = userPageAdClicks
         .partitionBy(UserPageAdClick::getUserId, upac -> upac,
-            KVSerde.of(new StringSerde(), new JsonSerdeV2<>(UserPageAdClick.class)), "userPageAdClicksByUserId")
-        .map(KV::getValue)
+            KVSerde.of(new StringSerde(), new JsonSerdeV2<>(UserPageAdClick.class)), "userPageAdClicksByUserId");
+
+    userPageAdClicksByUserId.map(KV::getValue)
         .window(Windows.keyedSessionWindow(UserPageAdClick::getUserId, Duration.ofSeconds(3),
             new StringSerde(), new JsonSerdeV2<>(UserPageAdClick.class)), "userAdClickWindow")
         .map(windowPane -> KV.of(windowPane.getKey().getKey(), String.valueOf(windowPane.getMessage().size())))
@@ -82,6 +104,16 @@ public class RepartitionJoinWindowApp implements StreamApplication {
             taskCoordinator.commit(TaskCoordinator.RequestScope.ALL_TASKS_IN_CONTAINER);
             messageCollector.send(new OutgoingMessageEnvelope(new SystemStream("kafka", outputTopic), null, message.getKey(), message.getValue()));
           });
+
+
+    intermediateStreams.add(((IntermediateMessageStreamImpl) pageViewsRepartitionedByViewId).getStreamSpec());
+    intermediateStreams.add(((IntermediateMessageStreamImpl) adClicksRepartitionedByViewId).getStreamSpec());
+    intermediateStreams.add(((IntermediateMessageStreamImpl) userPageAdClicksByUserId).getStreamSpec());
+
+  }
+
+  public List<StreamSpec> getIntermediateStreams() {
+    return intermediateStreams;
   }
 
   private static class UserPageViewAdClicksJoiner implements JoinFunction<String, PageView, AdClick, UserPageAdClick> {

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/test/java/org/apache/samza/test/operator/RepartitionWindowApp.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/RepartitionWindowApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/RepartitionWindowApp.java
new file mode 100644
index 0000000..e233793
--- /dev/null
+++ b/samza-test/src/test/java/org/apache/samza/test/operator/RepartitionWindowApp.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.test.operator;
+
+import java.time.Duration;
+import org.apache.samza.application.StreamApplication;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.KV;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.runtime.LocalApplicationRunner;
+import org.apache.samza.serializers.IntegerSerde;
+import org.apache.samza.serializers.JsonSerdeV2;
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.test.operator.data.PageView;
+import org.apache.samza.util.CommandLine;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link StreamApplication} that demonstrates a repartition followed by a windowed count.
+ */
+public class RepartitionWindowApp implements StreamApplication {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RepartitionWindowApp.class);
+
+  static final String INPUT_TOPIC = "page-views";
+  static final String OUTPUT_TOPIC = "Result";
+
+  public static void main(String[] args) {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+    RepartitionWindowApp reparApp = new RepartitionWindowApp();
+    LocalApplicationRunner runner = new LocalApplicationRunner(config);
+
+    runner.run(reparApp);
+    runner.waitForFinish();
+  }
+
+  @Override
+  public void init(StreamGraph graph, Config config) {
+
+    KVSerde<String, PageView>
+        pgeMsgSerde = KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageView.class));
+
+    graph.getInputStream(INPUT_TOPIC, pgeMsgSerde)
+        .map(KV::getValue)
+        .partitionBy(PageView::getUserId, m -> m, pgeMsgSerde, "p1")
+        .window(Windows.keyedSessionWindow(m -> m.getKey(), Duration.ofSeconds(3), () -> 0, (m, c) -> c + 1, new StringSerde("UTF-8"), new IntegerSerde()), "w1")
+        .map(wp -> KV.of(wp.getKey().getKey().toString(), String.valueOf(wp.getMessage())))
+        .sendTo(graph.getOutputStream(OUTPUT_TOPIC));
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/test/java/org/apache/samza/test/operator/SessionWindowApp.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/SessionWindowApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/SessionWindowApp.java
index 997127e..3224d24 100644
--- a/samza-test/src/test/java/org/apache/samza/test/operator/SessionWindowApp.java
+++ b/samza-test/src/test/java/org/apache/samza/test/operator/SessionWindowApp.java
@@ -19,6 +19,7 @@
 
 package org.apache.samza.test.operator;
 
+import java.time.Duration;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.Config;
 import org.apache.samza.operators.KV;
@@ -26,13 +27,15 @@ import org.apache.samza.operators.MessageStream;
 import org.apache.samza.operators.OutputStream;
 import org.apache.samza.operators.StreamGraph;
 import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.runtime.LocalApplicationRunner;
 import org.apache.samza.serializers.IntegerSerde;
 import org.apache.samza.serializers.JsonSerdeV2;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.StringSerde;
 import org.apache.samza.test.operator.data.PageView;
-
-import java.time.Duration;
+import org.apache.samza.util.CommandLine;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * A {@link StreamApplication} that demonstrates a filter followed by a session window.
@@ -40,10 +43,23 @@ import java.time.Duration;
 public class SessionWindowApp implements StreamApplication {
   private static final String INPUT_TOPIC = "page-views";
   private static final String OUTPUT_TOPIC = "page-view-counts";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SessionWindowApp.class);
   private static final String FILTER_KEY = "badKey";
 
+  public static void main(String[] args) {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+    SessionWindowApp app = new SessionWindowApp();
+    LocalApplicationRunner runner = new LocalApplicationRunner(config);
+
+    runner.run(app);
+    runner.waitForFinish();
+  }
+
   @Override
   public void init(StreamGraph graph, Config config) {
+
     MessageStream<PageView> pageViews = graph.getInputStream(INPUT_TOPIC, new JsonSerdeV2<>(PageView.class));
     OutputStream<KV<String, Integer>> outputStream =
         graph.getOutputStream(OUTPUT_TOPIC, new KVSerde<>(new StringSerde(), new IntegerSerde()));
@@ -54,5 +70,6 @@ public class SessionWindowApp implements StreamApplication {
             new StringSerde(), new JsonSerdeV2<>(PageView.class)), "sessionWindow")
         .map(m -> KV.of(m.getKey().getKey(), m.getMessage().size()))
         .sendTo(outputStream);
+
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java
index 77cd19a..5424888 100644
--- a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java
+++ b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java
@@ -122,7 +122,7 @@ public class TestRepartitionJoinWindowApp extends StreamApplicationIntegrationTe
 
     // Verify that messages in the intermediate stream will be deleted in 10 seconds
     long startTimeMs = System.currentTimeMillis();
-    for (StreamSpec spec: runner.getExecutionPlan(app).getIntermediateStreams()) {
+    for (StreamSpec spec: app.getIntermediateStreams()) {
       long remainingMessageNum = -1;
 
       while (remainingMessageNum != 0 && System.currentTimeMillis() - startTimeMs < 10000) {

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionWindowApp.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionWindowApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionWindowApp.java
new file mode 100644
index 0000000..fbc315f
--- /dev/null
+++ b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionWindowApp.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.operator;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.samza.config.JobCoordinatorConfig;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.config.TaskConfig;
+import org.apache.samza.test.operator.data.PageView;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.samza.test.operator.RepartitionWindowApp.*;
+
+/**
+ * Test driver for {@link RepartitionWindowApp}.
+ */
+public class TestRepartitionWindowApp extends StreamApplicationIntegrationTestHarness {
+
+  static final String APP_NAME = "PageViewCounterApp";
+
+  @Test
+  public void testRepartitionedSessionWindowCounter() throws Exception {
+    // create topics
+    createTopic(INPUT_TOPIC, 3);
+    createTopic(OUTPUT_TOPIC, 1);
+
+    // produce messages to different partitions.
+    ObjectMapper mapper = new ObjectMapper();
+    PageView pv = new PageView("india", "5.com", "userId1");
+    produceMessage(INPUT_TOPIC, 0, "userId1", mapper.writeValueAsString(pv));
+    pv = new PageView("china", "4.com", "userId2");
+    produceMessage(INPUT_TOPIC, 1, "userId2", mapper.writeValueAsString(pv));
+    pv = new PageView("india", "1.com", "userId1");
+    produceMessage(INPUT_TOPIC, 2, "userId1", mapper.writeValueAsString(pv));
+    pv = new PageView("india", "2.com", "userId1");
+    produceMessage(INPUT_TOPIC, 0, "userId1", mapper.writeValueAsString(pv));
+    pv = new PageView("india", "3.com", "userId1");
+    produceMessage(INPUT_TOPIC, 1, "userId1", mapper.writeValueAsString(pv));
+
+    Map<String, String> configs = new HashMap<>();
+    configs.put(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, "org.apache.samza.standalone.PassthroughJobCoordinatorFactory");
+    configs.put(JobCoordinatorConfig.JOB_COORDINATION_UTILS_FACTORY, "org.apache.samza.standalone.PassthroughCoordinationUtilsFactory");
+    configs.put(TaskConfig.GROUPER_FACTORY(), "org.apache.samza.container.grouper.task.GroupByContainerIdsFactory");
+    configs.put(String.format("streams.%s.samza.msg.serde", INPUT_TOPIC), "string");
+    configs.put(String.format("streams.%s.samza.key.serde", INPUT_TOPIC), "string");
+
+    // run the application
+    runApplication(new RepartitionWindowApp(), APP_NAME, new MapConfig(configs));
+
+    // consume and validate result
+    List<ConsumerRecord<String, String>> messages = consumeMessages(Collections.singletonList(OUTPUT_TOPIC), 2);
+    Assert.assertEquals(messages.size(), 2);
+
+    for (ConsumerRecord<String, String> message : messages) {
+      String key = message.key();
+      String value = message.value();
+      // Assert that there are 4 messages for userId1 and 1 message for userId2.
+      Assert.assertTrue(key.equals("userId1") || key.equals("userId2"));
+      if ("userId1".equals(key)) {
+        Assert.assertEquals(value, "4");
+      } else {
+        Assert.assertEquals(value, "1");
+      }
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/test/java/org/apache/samza/test/operator/TumblingWindowApp.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/TumblingWindowApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/TumblingWindowApp.java
index 5d2a17c..40a3f91 100644
--- a/samza-test/src/test/java/org/apache/samza/test/operator/TumblingWindowApp.java
+++ b/samza-test/src/test/java/org/apache/samza/test/operator/TumblingWindowApp.java
@@ -19,6 +19,7 @@
 
 package org.apache.samza.test.operator;
 
+import java.time.Duration;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.Config;
 import org.apache.samza.operators.KV;
@@ -26,13 +27,16 @@ import org.apache.samza.operators.MessageStream;
 import org.apache.samza.operators.OutputStream;
 import org.apache.samza.operators.StreamGraph;
 import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.runtime.LocalApplicationRunner;
 import org.apache.samza.serializers.IntegerSerde;
 import org.apache.samza.serializers.JsonSerdeV2;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.StringSerde;
 import org.apache.samza.test.operator.data.PageView;
+import org.apache.samza.util.CommandLine;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import java.time.Duration;
 
 /**
  * A {@link StreamApplication} that demonstrates a filter followed by a tumbling window.
@@ -40,10 +44,23 @@ import java.time.Duration;
 public class TumblingWindowApp implements StreamApplication {
   private static final String INPUT_TOPIC = "page-views";
   private static final String OUTPUT_TOPIC = "page-view-counts";
+
+  private static final Logger LOG = LoggerFactory.getLogger(TumblingWindowApp.class);
   private static final String FILTER_KEY = "badKey";
 
+  public static void main(String[] args) {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+    TumblingWindowApp app = new TumblingWindowApp();
+    LocalApplicationRunner runner = new LocalApplicationRunner(config);
+
+    runner.run(app);
+    runner.waitForFinish();
+  }
+
   @Override
   public void init(StreamGraph graph, Config config) {
+
     MessageStream<PageView> pageViews =
         graph.getInputStream(INPUT_TOPIC, new JsonSerdeV2<>(PageView.class));
     OutputStream<KV<String, Integer>> outputStream =
@@ -55,5 +72,6 @@ public class TumblingWindowApp implements StreamApplication {
             new StringSerde(), new JsonSerdeV2<>(PageView.class)), "tumblingWindow")
         .map(m -> KV.of(m.getKey().getKey(), m.getMessage().size()))
         .sendTo(outputStream);
+
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/test/java/org/apache/samza/test/operator/data/PageView.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/data/PageView.java b/samza-test/src/test/java/org/apache/samza/test/operator/data/PageView.java
index b114b43..e950366 100644
--- a/samza-test/src/test/java/org/apache/samza/test/operator/data/PageView.java
+++ b/samza-test/src/test/java/org/apache/samza/test/operator/data/PageView.java
@@ -19,9 +19,10 @@
 package org.apache.samza.test.operator.data;
 
 
+import java.io.Serializable;
 import org.codehaus.jackson.annotate.JsonProperty;
 
-public class PageView {
+public class PageView implements Serializable {
   private String viewId;
   private String pageId;
   private String userId;

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/test/java/org/apache/samza/test/processor/SharedContextFactories.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/processor/SharedContextFactories.java b/samza-test/src/test/java/org/apache/samza/test/processor/SharedContextFactories.java
new file mode 100644
index 0000000..9072bd2
--- /dev/null
+++ b/samza-test/src/test/java/org/apache/samza/test/processor/SharedContextFactories.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.processor;
+
+import java.util.HashMap;
+import java.util.Map;
+
+
+/**
+ * Shared context factories used in unit tests. This is a temporarily solution to enable sharing of test latches in different
+ * scope of context (i.e. in the container or the task). This is not intended for production usage.
+ */
+public class SharedContextFactories {
+
+  public static class SharedContextFactory {
+    static Map<String, SharedContextFactory> sharedContextFactories = new HashMap<>();
+
+    Map<String, Object> sharedObjects = new HashMap<>();
+
+    public Object getSharedObject(String resourceName) {
+      return this.sharedObjects.get(resourceName);
+    }
+
+    public void addSharedObject(String resourceName, Object sharedObj) {
+      this.sharedObjects.put(resourceName, sharedObj);
+    }
+
+    public static SharedContextFactory getInstance(String taskName) {
+      if (sharedContextFactories.get(taskName) == null) {
+        sharedContextFactories.putIfAbsent(taskName, new SharedContextFactory());
+      }
+      return sharedContextFactories.get(taskName);
+    }
+
+    public static void clearAll() {
+      sharedContextFactories.clear();
+    }
+  }
+
+  public static class ProcessorSharedContextFactory extends SharedContextFactory {
+    static Map<String, ProcessorSharedContextFactory> processorSharedFactories = new HashMap<>();
+
+    private final String processorId;
+
+    SharedContextFactory getTaskSharedContextFactory(String taskName) {
+      String globalTaskName = String.format("%s-%s", this.processorId, taskName);
+      return SharedContextFactory.getInstance(globalTaskName);
+    }
+
+    public static ProcessorSharedContextFactory getInstance(String processorId) {
+      if (processorSharedFactories.get(processorId) == null) {
+        processorSharedFactories.putIfAbsent(processorId, new ProcessorSharedContextFactory(processorId));
+      }
+      return processorSharedFactories.get(processorId);
+    }
+
+    ProcessorSharedContextFactory(String processorId) {
+      this.processorId = processorId;
+    }
+
+    public static void clearAll() {
+      processorSharedFactories.clear();
+    }
+  }
+
+  public static class GlobalSharedContextFactory extends SharedContextFactory {
+    static Map<String, GlobalSharedContextFactory> globalSharedContextFactories = new HashMap<>();
+
+    private final String appName;
+
+    GlobalSharedContextFactory(String appName) {
+      this.appName = appName;
+    }
+
+    ProcessorSharedContextFactory getProcessorSharedContextFactory(String processorName) {
+      String globalProcessorName = String.format("%s-%s", this.appName, processorName);
+      return ProcessorSharedContextFactory.getInstance(globalProcessorName);
+    }
+
+    public static GlobalSharedContextFactory getInstance(String appName) {
+      if (globalSharedContextFactories.get(appName) == null) {
+        globalSharedContextFactories.putIfAbsent(appName, new GlobalSharedContextFactory(appName));
+      }
+      return globalSharedContextFactories.get(appName);
+    }
+
+    public static void clearAll() {
+      globalSharedContextFactories.clear();
+    }
+  }
+
+  public static GlobalSharedContextFactory getGlobalSharedContextFactory(String appName) {
+    return GlobalSharedContextFactory.getInstance(appName);
+  }
+
+  public static void clearAll() {
+    GlobalSharedContextFactory.clearAll();
+    ProcessorSharedContextFactory.clearAll();
+    SharedContextFactory.clearAll();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/test/java/org/apache/samza/test/processor/TestStreamApplication.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/processor/TestStreamApplication.java b/samza-test/src/test/java/org/apache/samza/test/processor/TestStreamApplication.java
new file mode 100644
index 0000000..db12351
--- /dev/null
+++ b/samza-test/src/test/java/org/apache/samza/test/processor/TestStreamApplication.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.processor;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.Serializable;
+import java.util.concurrent.CountDownLatch;
+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.operators.MessageStream;
+import org.apache.samza.operators.OutputStream;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.functions.MapFunction;
+import org.apache.samza.serializers.NoOpSerde;
+import org.apache.samza.serializers.StringSerde;
+
+
+/**
+ * Test class to create an {@link StreamApplication} instance
+ */
+public class TestStreamApplication implements StreamApplication, Serializable {
+
+  private final String inputTopic;
+  private final String outputTopic;
+  private final String appName;
+  private final String processorName;
+
+  private TestStreamApplication(String inputTopic, String outputTopic, String appName, String processorName) {
+    this.inputTopic = inputTopic;
+    this.outputTopic = outputTopic;
+    this.appName = appName;
+    this.processorName = processorName;
+  }
+
+  @Override
+  public void init(StreamGraph graph, Config config) {
+    MessageStream<String> inputStream = graph.getInputStream(inputTopic, new NoOpSerde<String>());
+    OutputStream<String> outputStream = graph.getOutputStream(outputTopic, new StringSerde());
+    inputStream.map(new MapFunction<String, String>() {
+      transient CountDownLatch latch1;
+      transient CountDownLatch latch2;
+      transient StreamApplicationCallback callback;
+
+      @Override
+      public String apply(String message) {
+        TestKafkaEvent incomingMessage = TestKafkaEvent.fromString(message);
+        if (callback != null) {
+          callback.onMessage(incomingMessage);
+        }
+        if (latch1 != null) {
+          latch1.countDown();
+        }
+        if (latch2 != null) {
+          latch2.countDown();
+        }
+        return incomingMessage.toString();
+      }
+
+      private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+        in.defaultReadObject();
+        SharedContextFactories.SharedContextFactory contextFactory =
+            SharedContextFactories.getGlobalSharedContextFactory(appName).getProcessorSharedContextFactory(processorName);
+        this.latch1 = (CountDownLatch) contextFactory.getSharedObject("processedMsgLatch");
+        this.latch2 = (CountDownLatch) contextFactory.getSharedObject("kafkaMsgsConsumedLatch");
+        this.callback = (StreamApplicationCallback) contextFactory.getSharedObject("callback");
+      }
+    }).sendTo(outputStream);
+  }
+
+  public interface StreamApplicationCallback {
+    void onMessage(TestKafkaEvent m);
+  }
+
+  public static class TestKafkaEvent implements Serializable {
+
+    // Actual content of the event.
+    private String eventData;
+
+    // Contains Integer value, which is greater than previous message id.
+    private String eventId;
+
+    TestKafkaEvent(String eventId, String eventData) {
+      this.eventData = eventData;
+      this.eventId = eventId;
+    }
+
+    String getEventId() {
+      return eventId;
+    }
+
+    String getEventData() {
+      return eventData;
+    }
+
+    @Override
+    public String toString() {
+      return eventId + "|" + eventData;
+    }
+
+    static TestKafkaEvent fromString(String message) {
+      String[] messageComponents = message.split("|");
+      return new TestKafkaEvent(messageComponents[0], messageComponents[1]);
+    }
+  }
+
+  public static StreamApplication getInstance(
+      String inputTopic,
+      String outputTopic,
+      CountDownLatch processedMessageLatch,
+      StreamApplicationCallback callback,
+      CountDownLatch kafkaEventsConsumedLatch,
+      Config config) {
+    String appName = String.format("%s-%s", config.get(ApplicationConfig.APP_NAME), config.get(ApplicationConfig.APP_ID));
+    String processorName = config.get(JobConfig.PROCESSOR_ID());
+    registerLatches(processedMessageLatch, kafkaEventsConsumedLatch, callback, appName, processorName);
+
+    StreamApplication app = new TestStreamApplication(inputTopic, outputTopic, appName, processorName);
+    return app;
+  }
+
+  private static void registerLatches(CountDownLatch processedMessageLatch, CountDownLatch kafkaEventsConsumedLatch,
+      StreamApplicationCallback callback, String appName, String processorName) {
+    SharedContextFactories.SharedContextFactory contextFactory = SharedContextFactories.getGlobalSharedContextFactory(appName).getProcessorSharedContextFactory(processorName);
+    contextFactory.addSharedObject("processedMsgLatch", processedMessageLatch);
+    contextFactory.addSharedObject("kafkaMsgsConsumedLatch", kafkaEventsConsumedLatch);
+    contextFactory.addSharedObject("callback", callback);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java
index 9d2cd92..0b0a271 100644
--- a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java
+++ b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java
@@ -23,6 +23,14 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
 import kafka.admin.AdminUtils;
 import kafka.utils.TestUtils;
 import org.I0Itec.zkclient.ZkClient;
@@ -42,33 +50,23 @@ import org.apache.samza.container.TaskName;
 import org.apache.samza.job.ApplicationStatus;
 import org.apache.samza.job.model.JobModel;
 import org.apache.samza.job.model.TaskModel;
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.OutputStream;
-import org.apache.samza.operators.StreamGraph;
 import org.apache.samza.runtime.LocalApplicationRunner;
-import org.apache.samza.serializers.NoOpSerde;
-import org.apache.samza.serializers.StringSerde;
 import org.apache.samza.test.StandaloneIntegrationTestHarness;
 import org.apache.samza.test.StandaloneTestUtils;
+import org.apache.samza.test.processor.TestStreamApplication.StreamApplicationCallback;
+import org.apache.samza.test.processor.TestStreamApplication.TestKafkaEvent;
 import org.apache.samza.util.NoOpMetricsRegistry;
 import org.apache.samza.zk.ZkJobCoordinatorFactory;
 import org.apache.samza.zk.ZkKeyBuilder;
 import org.apache.samza.zk.ZkUtils;
-import org.junit.*;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
 import org.junit.rules.ExpectedException;
 import org.junit.rules.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.UUID;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicBoolean;
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -102,9 +100,6 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne
   private ApplicationConfig applicationConfig1;
   private ApplicationConfig applicationConfig2;
   private ApplicationConfig applicationConfig3;
-  private LocalApplicationRunner applicationRunner1;
-  private LocalApplicationRunner applicationRunner2;
-  private LocalApplicationRunner applicationRunner3;
   private String testStreamAppName;
   private String testStreamAppId;
 
@@ -141,11 +136,6 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne
     zkUtils = new ZkUtils(zkKeyBuilder, zkClient, ZK_CONNECTION_TIMEOUT_MS, ZK_SESSION_TIMEOUT_MS, new NoOpMetricsRegistry());
     zkUtils.connect();
 
-    // Create local application runners.
-    applicationRunner1 = new LocalApplicationRunner(applicationConfig1);
-    applicationRunner2 = new LocalApplicationRunner(applicationConfig2);
-    applicationRunner3 = new LocalApplicationRunner(applicationConfig3);
-
     for (String kafkaTopic : ImmutableList.of(inputKafkaTopic, outputKafkaTopic)) {
       LOGGER.info("Creating kafka topic: {}.", kafkaTopic);
       TestUtils.createTopic(zkUtils(), kafkaTopic, 5, 1, servers(), new Properties());
@@ -251,9 +241,10 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne
 
     // Set up stream app 2.
     CountDownLatch processedMessagesLatch = new CountDownLatch(NUM_KAFKA_EVENTS);
-    LocalApplicationRunner localApplicationRunner2 = new LocalApplicationRunner(new MapConfig(applicationConfig2, testConfig));
-    StreamApplication streamApp2 = new TestStreamApplication(inputSinglePartitionKafkaTopic, outputSinglePartitionKafkaTopic,
-        processedMessagesLatch, null, null);
+    Config testAppConfig2 = new MapConfig(applicationConfig2, testConfig);
+    LocalApplicationRunner localApplicationRunner2 = new LocalApplicationRunner(testAppConfig2);
+    StreamApplication streamApp2 = TestStreamApplication.getInstance(inputSinglePartitionKafkaTopic, outputSinglePartitionKafkaTopic,
+        processedMessagesLatch, null, null, testAppConfig2);
 
     // Callback handler for streamApp1.
     StreamApplicationCallback streamApplicationCallback = message -> {
@@ -272,9 +263,10 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne
     CountDownLatch kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS * 2);
 
     // Set up stream app 1.
-    LocalApplicationRunner localApplicationRunner1 = new LocalApplicationRunner(new MapConfig(applicationConfig1, testConfig));
-    StreamApplication streamApp1 = new TestStreamApplication(inputSinglePartitionKafkaTopic, outputSinglePartitionKafkaTopic,
-        null, streamApplicationCallback, kafkaEventsConsumedLatch);
+    Config testAppConfig1 = new MapConfig(applicationConfig1, testConfig);
+    LocalApplicationRunner localApplicationRunner1 = new LocalApplicationRunner(testAppConfig1);
+    StreamApplication streamApp1 = TestStreamApplication.getInstance(inputSinglePartitionKafkaTopic, outputSinglePartitionKafkaTopic,
+        null, streamApplicationCallback, kafkaEventsConsumedLatch, testAppConfig1);
     localApplicationRunner1.run(streamApp1);
 
     kafkaEventsConsumedLatch.await();
@@ -288,6 +280,13 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne
     // TODO: After SAMZA-1364 add assertion for localApplicationRunner2.status(streamApp)
     // ProcessedMessagesLatch shouldn't have changed. Should retain it's initial value.
     assertEquals(NUM_KAFKA_EVENTS, processedMessagesLatch.getCount());
+
+    // TODO: re-enable the following kill and waitForFinish calls after fixing SAMZA-1665
+    // localApplicationRunner1.kill(streamApp1);
+    // localApplicationRunner2.kill(streamApp2);
+
+    // localApplicationRunner1.waitForFinish();
+    // localApplicationRunner2.waitForFinish();
   }
 
   /**
@@ -326,8 +325,9 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne
 
     // Set up streamApp2.
     CountDownLatch processedMessagesLatch = new CountDownLatch(NUM_KAFKA_EVENTS * 2);
-    LocalApplicationRunner localApplicationRunner2 = new LocalApplicationRunner(new MapConfig(applicationConfig2, testConfig));
-    StreamApplication streamApp2 = new TestStreamApplication(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch, null, null);
+    Config testAppConfig2 = new MapConfig(applicationConfig2, testConfig);
+    LocalApplicationRunner localApplicationRunner2 = new LocalApplicationRunner(testAppConfig2);
+    StreamApplication streamApp2 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch, null, null, testAppConfig2);
 
     // Callback handler for streamApp1.
     StreamApplicationCallback streamApplicationCallback = message -> {
@@ -348,9 +348,10 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne
     CountDownLatch kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS * 2 + 1);
 
     // Set up stream app 1.
-    LocalApplicationRunner localApplicationRunner1 = new LocalApplicationRunner(new MapConfig(applicationConfig1, testConfig));
-    StreamApplication streamApp1 = new TestStreamApplication(inputKafkaTopic, outputKafkaTopic, null,
-        streamApplicationCallback, kafkaEventsConsumedLatch);
+    Config testAppConfig1 = new MapConfig(applicationConfig1, testConfig);
+    LocalApplicationRunner localApplicationRunner1 = new LocalApplicationRunner(testAppConfig1);
+    StreamApplication streamApp1 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, null,
+        streamApplicationCallback, kafkaEventsConsumedLatch, testAppConfig1);
     localApplicationRunner1.run(streamApp1);
 
     kafkaEventsConsumedLatch.await();
@@ -381,8 +382,16 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne
     assertEquals(taskModel1.getSystemStreamPartitions(), taskModel2.getSystemStreamPartitions());
     assertTrue(!taskModel1.getTaskName().getTaskName().equals(taskModel2.getTaskName().getTaskName()));
 
-    // TODO: After SAMZA-1364 add assertion for localApplicationRunner2.status(streamApp)
     processedMessagesLatch.await();
+
+    assertEquals(ApplicationStatus.Running, localApplicationRunner2.status(streamApp2));
+
+    // TODO: re-enable the following kill and waitForFinish calls after fixing SAMZA-1665
+    // localApplicationRunner1.kill(streamApp1);
+    // localApplicationRunner2.kill(streamApp2);
+
+    // localApplicationRunner1.waitForFinish();
+    // localApplicationRunner2.waitForFinish();
   }
 
   @Test
@@ -396,8 +405,13 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne
     CountDownLatch processedMessagesLatch2 = new CountDownLatch(1);
     CountDownLatch processedMessagesLatch3 = new CountDownLatch(1);
 
-    StreamApplication streamApp1 = new TestStreamApplication(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch1, null, kafkaEventsConsumedLatch);
-    StreamApplication streamApp2 = new TestStreamApplication(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch2, null, kafkaEventsConsumedLatch);
+    StreamApplication streamApp1 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch1, null, kafkaEventsConsumedLatch, applicationConfig1);
+    StreamApplication streamApp2 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch2, null, kafkaEventsConsumedLatch, applicationConfig2);
+    StreamApplication streamApp3 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch3, null, kafkaEventsConsumedLatch, applicationConfig3);
+
+    // Create LocalApplicationRunners
+    LocalApplicationRunner applicationRunner1 = new LocalApplicationRunner(applicationConfig1);
+    LocalApplicationRunner applicationRunner2 = new LocalApplicationRunner(applicationConfig2);
 
     // Run stream applications.
     applicationRunner1.run(streamApp1);
@@ -428,7 +442,7 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne
     kafkaEventsConsumedLatch.await();
     publishKafkaEvents(inputKafkaTopic, 0, 2 * NUM_KAFKA_EVENTS, PROCESSOR_IDS[0]);
 
-    StreamApplication streamApp3 = new TestStreamApplication(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch3, null, kafkaEventsConsumedLatch);
+    LocalApplicationRunner applicationRunner3 = new LocalApplicationRunner(applicationConfig3);
     applicationRunner3.run(streamApp3);
     processedMessagesLatch3.await();
 
@@ -441,6 +455,13 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne
     jobModel = zkUtils.getJobModel(jobModelVersion);
     assertEquals(Sets.newHashSet("0000000001", "0000000002"), jobModel.getContainers().keySet());
     assertEquals(2, jobModel.getContainers().size());
+
+    // TODO: re-enable the following kill and waitForFinish calls after fixing SAMZA-1665
+    // applicationRunner2.kill(streamApp2);
+    // applicationRunner3.kill(streamApp3);
+
+    // applicationRunner2.waitForFinish();
+    // applicationRunner3.waitForFinish();
   }
 
   @Test
@@ -453,8 +474,12 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne
     CountDownLatch processedMessagesLatch1 = new CountDownLatch(1);
     CountDownLatch processedMessagesLatch2 = new CountDownLatch(1);
 
-    StreamApplication streamApp1 = new TestStreamApplication(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch1, null, kafkaEventsConsumedLatch);
-    StreamApplication streamApp2 = new TestStreamApplication(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch2, null, kafkaEventsConsumedLatch);
+    StreamApplication streamApp1 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch1, null, kafkaEventsConsumedLatch, applicationConfig1);
+    StreamApplication streamApp2 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch2, null, kafkaEventsConsumedLatch, applicationConfig2);
+
+    // Create LocalApplicationRunners
+    LocalApplicationRunner applicationRunner1 = new LocalApplicationRunner(applicationConfig1);
+    LocalApplicationRunner applicationRunner2 = new LocalApplicationRunner(applicationConfig2);
 
     // Run stream applications.
     applicationRunner1.run(streamApp1);
@@ -464,15 +489,24 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne
     processedMessagesLatch1.await();
     processedMessagesLatch2.await();
 
-    LocalApplicationRunner applicationRunner3 = new LocalApplicationRunner(new MapConfig(applicationConfig2));
+    LocalApplicationRunner applicationRunner3 = new LocalApplicationRunner(applicationConfig2);
 
     // Create a stream app with same processor id as SP2 and run it. It should fail.
     publishKafkaEvents(inputKafkaTopic, NUM_KAFKA_EVENTS, 2 * NUM_KAFKA_EVENTS, PROCESSOR_IDS[2]);
     kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS);
-    StreamApplication streamApp3 = new TestStreamApplication(inputKafkaTopic, outputKafkaTopic, null, null, kafkaEventsConsumedLatch);
+    StreamApplication streamApp3 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, null, null, kafkaEventsConsumedLatch, applicationConfig2);
     // Fail when the duplicate processor joins.
     expectedException.expect(SamzaException.class);
-    applicationRunner3.run(streamApp3);
+    try {
+      applicationRunner3.run(streamApp3);
+    } finally {
+      // TODO: re-enable the following kill and waitForFinish calls after fixing SAMZA-1665
+      // applicationRunner1.kill(streamApp1);
+      // applicationRunner2.kill(streamApp2);
+
+      // applicationRunner1.waitForFinish();
+      // applicationRunner2.waitForFinish();
+    }
   }
 
   @Test
@@ -496,13 +530,16 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne
     LocalApplicationRunner applicationRunner1 = new LocalApplicationRunner(applicationConfig1);
     LocalApplicationRunner applicationRunner2 = new LocalApplicationRunner(applicationConfig2);
 
+    List<TestKafkaEvent> messagesProcessed = new ArrayList<>();
+    StreamApplicationCallback streamApplicationCallback = m -> messagesProcessed.add(m);
+
     // Create StreamApplication from configuration.
     CountDownLatch kafkaEventsConsumedLatch = new CountDownLatch(NUM_KAFKA_EVENTS);
     CountDownLatch processedMessagesLatch1 = new CountDownLatch(1);
     CountDownLatch processedMessagesLatch2 = new CountDownLatch(1);
 
-    StreamApplication streamApp1 = new TestStreamApplication(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch1, null, kafkaEventsConsumedLatch);
-    StreamApplication streamApp2 = new TestStreamApplication(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch2, null, kafkaEventsConsumedLatch);
+    StreamApplication streamApp1 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch1, streamApplicationCallback, kafkaEventsConsumedLatch, applicationConfig1);
+    StreamApplication streamApp2 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch2, null, kafkaEventsConsumedLatch, applicationConfig2);
 
     // Run stream application.
     applicationRunner1.run(streamApp1);
@@ -521,7 +558,7 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne
     LocalApplicationRunner applicationRunner4 = new LocalApplicationRunner(applicationConfig1);
     processedMessagesLatch1 = new CountDownLatch(1);
     publishKafkaEvents(inputKafkaTopic, NUM_KAFKA_EVENTS, 2 * NUM_KAFKA_EVENTS, PROCESSOR_IDS[0]);
-    streamApp1 = new TestStreamApplication(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch1, null, kafkaEventsConsumedLatch);
+    streamApp1 = TestStreamApplication.getInstance(inputKafkaTopic, outputKafkaTopic, processedMessagesLatch1, streamApplicationCallback, kafkaEventsConsumedLatch, applicationConfig1);
     applicationRunner4.run(streamApp1);
 
     processedMessagesLatch1.await();
@@ -532,85 +569,13 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne
 
     assertEquals(Integer.parseInt(jobModelVersion) + 1, Integer.parseInt(newJobModelVersion));
     assertEquals(jobModel.getContainers(), newJobModel.getContainers());
-  }
-
-  public interface StreamApplicationCallback {
-    void onMessageReceived(TestKafkaEvent message);
-  }
-
-  private static class TestKafkaEvent implements Serializable {
-
-    // Actual content of the event.
-    private String eventData;
-
-    // Contains Integer value, which is greater than previous message id.
-    private String eventId;
 
-    TestKafkaEvent(String eventId, String eventData) {
-      this.eventData = eventData;
-      this.eventId = eventId;
-    }
-
-    String getEventId() {
-      return eventId;
-    }
-
-    String getEventData() {
-      return eventData;
-    }
-
-    @Override
-    public String toString() {
-      return eventId + "|" + eventData;
-    }
+    // TODO: re-enable the following kill and waitForFinish calls after fixing SAMZA-1665
+    // applicationRunner2.kill(streamApp2);
+    // applicationRunner4.kill(streamApp1);
 
-    static TestKafkaEvent fromString(String message) {
-      String[] messageComponents = message.split("\\|");
-      return new TestKafkaEvent(messageComponents[0], messageComponents[1]);
-    }
+    // applicationRunner2.waitForFinish();
+    // applicationRunner4.waitForFinish();
   }
 
-  /**
-   * Publishes all input events to output topic(has no processing logic)
-   * and triggers {@link StreamApplicationCallback} with each received event.
-   **/
-  private static class TestStreamApplication implements StreamApplication {
-
-    private final String inputTopic;
-    private final String outputTopic;
-    private final CountDownLatch processedMessagesLatch;
-    private final StreamApplicationCallback streamApplicationCallback;
-    private final CountDownLatch kafkaEventsConsumedLatch;
-
-    TestStreamApplication(String inputTopic, String outputTopic,
-        CountDownLatch processedMessagesLatch,
-        StreamApplicationCallback streamApplicationCallback, CountDownLatch kafkaEventsConsumedLatch) {
-      this.inputTopic = inputTopic;
-      this.outputTopic = outputTopic;
-      this.processedMessagesLatch = processedMessagesLatch;
-      this.streamApplicationCallback = streamApplicationCallback;
-      this.kafkaEventsConsumedLatch = kafkaEventsConsumedLatch;
-    }
-
-    @Override
-    public void init(StreamGraph graph, Config config) {
-      MessageStream<String> inputStream = graph.getInputStream(inputTopic, new NoOpSerde<String>());
-      OutputStream<String> outputStream = graph.getOutputStream(outputTopic, new StringSerde());
-      inputStream
-          .map(msg -> {
-              TestKafkaEvent incomingMessage = TestKafkaEvent.fromString((String) msg);
-              if (streamApplicationCallback != null) {
-                streamApplicationCallback.onMessageReceived(incomingMessage);
-              }
-              if (processedMessagesLatch != null) {
-                processedMessagesLatch.countDown();
-              }
-              if (kafkaEventsConsumedLatch != null) {
-                kafkaEventsConsumedLatch.countDown();
-              }
-              return incomingMessage.toString();
-            })
-          .sendTo(outputStream);
-    }
-  }
 }


[04/10] samza git commit: SAMZA-1659: Serializable OperatorSpec

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/main/java/org/apache/samza/sql/translator/ProjectTranslator.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/ProjectTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/ProjectTranslator.java
index a0bd45f..8e6f687 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/translator/ProjectTranslator.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/ProjectTranslator.java
@@ -31,9 +31,12 @@ import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
 import org.apache.samza.SamzaException;
+import org.apache.samza.config.Config;
 import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.functions.MapFunction;
 import org.apache.samza.sql.data.Expression;
 import org.apache.samza.sql.data.SamzaSqlRelMessage;
+import org.apache.samza.task.TaskContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -46,37 +49,37 @@ class ProjectTranslator {
 
   private static final Logger LOG = LoggerFactory.getLogger(ProjectTranslator.class);
 
-  void translate(final Project project, final TranslatorContext context) {
-    MessageStream<SamzaSqlRelMessage> messageStream = context.getMessageStream(project.getInput().getId());
-    List<Integer> flattenProjects =
-        project.getProjects().stream().filter(this::isFlatten).map(this::getProjectIndex).collect(Collectors.toList());
+  private static class ProjectMapFunction implements MapFunction<SamzaSqlRelMessage, SamzaSqlRelMessage> {
+    private transient Project project;
+    private transient Expression expr;
+    private transient TranslatorContext context;
 
-    if (flattenProjects.size() > 0) {
-      if (flattenProjects.size() > 1) {
-        String msg = "Multiple flatten operators in a single query is not supported";
-        LOG.error(msg);
-        throw new SamzaException(msg);
-      }
+    private final int projectId;
 
-      messageStream = translateFlatten(flattenProjects.get(0), messageStream);
+    ProjectMapFunction(int projectId) {
+      this.projectId = projectId;
     }
 
-    Expression expr = context.getExpressionCompiler().compile(project.getInputs(), project.getProjects());
+    @Override
+    public void init(Config config, TaskContext taskContext) {
+      this.context = (TranslatorContext) taskContext.getUserContext();
+      this.project = (Project) this.context.getRelNode(projectId);
+      this.expr = this.context.getExpressionCompiler().compile(project.getInputs(), project.getProjects());
+    }
 
-    MessageStream<SamzaSqlRelMessage> outputStream = messageStream.map(m -> {
+    @Override
+    public SamzaSqlRelMessage apply(SamzaSqlRelMessage message) {
       RelDataType type = project.getRowType();
       Object[] output = new Object[type.getFieldCount()];
       expr.execute(context.getExecutionContext(), context.getDataContext(),
-          m.getSamzaSqlRelRecord().getFieldValues().toArray(), output);
+          message.getSamzaSqlRelRecord().getFieldValues().toArray(), output);
       List<String> names = new ArrayList<>();
       for (int index = 0; index < output.length; index++) {
         names.add(index, project.getNamedProjects().get(index).getValue());
       }
 
       return new SamzaSqlRelMessage(names, Arrays.asList(output));
-    });
-
-    context.registerMessageStream(project.getId(), outputStream);
+    }
   }
 
   private MessageStream<SamzaSqlRelMessage> translateFlatten(Integer flattenIndex,
@@ -106,4 +109,27 @@ class ProjectTranslator {
   private Integer getProjectIndex(RexNode rexNode) {
     return ((RexInputRef) ((RexCall) rexNode).getOperands().get(0)).getIndex();
   }
+
+  void translate(final Project project, final TranslatorContext context) {
+    MessageStream<SamzaSqlRelMessage> messageStream = context.getMessageStream(project.getInput().getId());
+    List<Integer> flattenProjects =
+        project.getProjects().stream().filter(this::isFlatten).map(this::getProjectIndex).collect(Collectors.toList());
+
+    if (flattenProjects.size() > 0) {
+      if (flattenProjects.size() > 1) {
+        String msg = "Multiple flatten operators in a single query is not supported";
+        LOG.error(msg);
+        throw new SamzaException(msg);
+      }
+      messageStream = translateFlatten(flattenProjects.get(0), messageStream);
+    }
+
+    final int projectId = project.getId();
+
+    MessageStream<SamzaSqlRelMessage> outputStream = messageStream.map(new ProjectMapFunction(projectId));
+
+    context.registerMessageStream(project.getId(), outputStream);
+    context.registerRelNode(project.getId(), project);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java
index eda73a7..1db3000 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java
@@ -19,6 +19,7 @@
 
 package org.apache.samza.sql.translator;
 
+import java.util.Map;
 import java.util.Optional;
 
 import org.apache.calcite.rel.RelNode;
@@ -29,11 +30,14 @@ import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.ContextManager;
 import org.apache.samza.SamzaException;
 import org.apache.samza.operators.KV;
 import org.apache.samza.operators.MessageStream;
 import org.apache.samza.operators.MessageStreamImpl;
 import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.functions.MapFunction;
 import org.apache.samza.operators.TableDescriptor;
 import org.apache.samza.sql.data.SamzaSqlExecutionContext;
 import org.apache.samza.sql.data.SamzaSqlRelMessage;
@@ -43,6 +47,7 @@ import org.apache.samza.sql.interfaces.SqlIOConfig;
 import org.apache.samza.sql.planner.QueryPlanner;
 import org.apache.samza.sql.runner.SamzaSqlApplicationConfig;
 import org.apache.samza.sql.testutil.SamzaSqlQueryParser;
+import org.apache.samza.task.TaskContext;
 import org.apache.samza.table.Table;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -58,11 +63,33 @@ public class QueryTranslator {
 
   private final ScanTranslator scanTranslator;
   private final SamzaSqlApplicationConfig sqlConfig;
+  private final Map<String, SamzaRelConverter> converters;
+
+  private static class OutputMapFunction implements MapFunction<SamzaSqlRelMessage, KV<Object, Object>> {
+    private transient SamzaRelConverter samzaMsgConverter;
+    private final String outputTopic;
+
+    OutputMapFunction(String outputTopic) {
+      this.outputTopic = outputTopic;
+    }
+
+    @Override
+    public void init(Config config, TaskContext taskContext) {
+      TranslatorContext context = (TranslatorContext) taskContext.getUserContext();
+      this.samzaMsgConverter = context.getMsgConverter(outputTopic);
+    }
+
+    @Override
+    public KV<Object, Object> apply(SamzaSqlRelMessage message) {
+      return this.samzaMsgConverter.convertToSamzaMessage(message);
+    }
+  }
 
   public QueryTranslator(SamzaSqlApplicationConfig sqlConfig) {
     this.sqlConfig = sqlConfig;
     scanTranslator =
         new ScanTranslator(sqlConfig.getSamzaRelConverters(), sqlConfig.getInputSystemStreamConfigBySource());
+    this.converters = sqlConfig.getSamzaRelConverters();
   }
 
   public void translate(SamzaSqlQueryParser.QueryInfo queryInfo, StreamGraph streamGraph) {
@@ -71,7 +98,7 @@ public class QueryTranslator {
             sqlConfig.getUdfMetadata());
     final SamzaSqlExecutionContext executionContext = new SamzaSqlExecutionContext(this.sqlConfig);
     final RelRoot relRoot = planner.plan(queryInfo.getSelectQuery());
-    final TranslatorContext context = new TranslatorContext(streamGraph, relRoot, executionContext);
+    final TranslatorContext context = new TranslatorContext(streamGraph, relRoot, executionContext, this.converters);
     final RelNode node = relRoot.project();
     final SqlIOResolver ioResolver = context.getExecutionContext().getSamzaSqlApplicationConfig().getIoResolver();
 
@@ -119,9 +146,8 @@ public class QueryTranslator {
 
     String sink = queryInfo.getSink();
     SqlIOConfig sinkConfig = sqlConfig.getOutputSystemStreamConfigsBySource().get(sink);
-    SamzaRelConverter samzaMsgConverter = sqlConfig.getSamzaRelConverters().get(queryInfo.getSink());
     MessageStreamImpl<SamzaSqlRelMessage> stream = (MessageStreamImpl<SamzaSqlRelMessage>) context.getMessageStream(node.getId());
-    MessageStream<KV<Object, Object>> outputStream = stream.map(samzaMsgConverter::convertToSamzaMessage);
+    MessageStream<KV<Object, Object>> outputStream = stream.map(new OutputMapFunction(sink));
 
     Optional<TableDescriptor> tableDescriptor = sinkConfig.getTableDescriptor();
     if (!tableDescriptor.isPresent()) {
@@ -135,5 +161,19 @@ public class QueryTranslator {
       }
       outputStream.sendTo(outputTable);
     }
+
+    streamGraph.withContextManager(new ContextManager() {
+      @Override
+      public void init(Config config, TaskContext taskContext) {
+        taskContext.setUserContext(context.clone());
+      }
+
+      @Override
+      public void close() {
+
+      }
+
+    });
+
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/main/java/org/apache/samza/sql/translator/SamzaSqlRelMessageJoinFunction.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/SamzaSqlRelMessageJoinFunction.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/SamzaSqlRelMessageJoinFunction.java
index df88a7c..889ea97 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/translator/SamzaSqlRelMessageJoinFunction.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/SamzaSqlRelMessageJoinFunction.java
@@ -44,20 +44,20 @@ public class SamzaSqlRelMessageJoinFunction
 
   private final JoinRelType joinRelType;
   private final boolean isTablePosOnRight;
-  private final List<Integer> streamFieldIds;
+  private final ArrayList<Integer> streamFieldIds;
   // Table field names are used in the outer join when the table record is not found.
-  private final List<String> tableFieldNames;
-  private final List<String> outFieldNames;
+  private final ArrayList<String> tableFieldNames;
+  private final ArrayList<String> outFieldNames;
 
-  public SamzaSqlRelMessageJoinFunction(JoinRelType joinRelType, boolean isTablePosOnRight,
+  SamzaSqlRelMessageJoinFunction(JoinRelType joinRelType, boolean isTablePosOnRight,
       List<Integer> streamFieldIds, List<String> streamFieldNames, List<String> tableFieldNames) {
     this.joinRelType = joinRelType;
     this.isTablePosOnRight = isTablePosOnRight;
     Validate.isTrue((joinRelType.compareTo(JoinRelType.LEFT) == 0 && isTablePosOnRight) ||
         (joinRelType.compareTo(JoinRelType.RIGHT) == 0 && !isTablePosOnRight) ||
         joinRelType.compareTo(JoinRelType.INNER) == 0);
-    this.streamFieldIds = streamFieldIds;
-    this.tableFieldNames = tableFieldNames;
+    this.streamFieldIds = new ArrayList<>(streamFieldIds);
+    this.tableFieldNames = new ArrayList<>(tableFieldNames);
     this.outFieldNames = new ArrayList<>();
     if (isTablePosOnRight) {
       outFieldNames.addAll(streamFieldNames);

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java
index 1f9ed52..fa3d9d3 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java
@@ -23,11 +23,14 @@ import java.util.List;
 import java.util.Map;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.commons.lang.Validate;
+import org.apache.samza.config.Config;
 import org.apache.samza.operators.KV;
 import org.apache.samza.operators.MessageStream;
 import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.functions.MapFunction;
 import org.apache.samza.sql.data.SamzaSqlRelMessage;
 import org.apache.samza.sql.interfaces.SamzaRelConverter;
+import org.apache.samza.task.TaskContext;
 import org.apache.samza.sql.interfaces.SqlIOConfig;
 
 
@@ -45,17 +48,36 @@ class ScanTranslator {
     this.systemStreamConfig = ssc;
   }
 
+  private static class ScanMapFunction implements MapFunction<KV<Object, Object>, SamzaSqlRelMessage> {
+    private transient SamzaRelConverter msgConverter;
+    private final String streamName;
+
+    ScanMapFunction(String sourceStreamName) {
+      this.streamName = sourceStreamName;
+    }
+
+    @Override
+    public void init(Config config, TaskContext taskContext) {
+      TranslatorContext context = (TranslatorContext) taskContext.getUserContext();
+      this.msgConverter = context.getMsgConverter(streamName);
+    }
+
+    @Override
+    public SamzaSqlRelMessage apply(KV<Object, Object> message) {
+      return this.msgConverter.convertToRelMessage(message);
+    }
+  }
+
   void translate(final TableScan tableScan, final TranslatorContext context) {
     StreamGraph streamGraph = context.getStreamGraph();
     List<String> tableNameParts = tableScan.getTable().getQualifiedName();
     String sourceName = SqlIOConfig.getSourceFromSourceParts(tableNameParts);
 
     Validate.isTrue(relMsgConverters.containsKey(sourceName), String.format("Unknown source %s", sourceName));
-    SamzaRelConverter converter = relMsgConverters.get(sourceName);
-    String streamName = systemStreamConfig.get(sourceName).getStreamName();
+    final String streamName = systemStreamConfig.get(sourceName).getStreamName();
 
     MessageStream<KV<Object, Object>> inputStream = streamGraph.getInputStream(streamName);
-    MessageStream<SamzaSqlRelMessage> samzaSqlRelMessageStream = inputStream.map(converter::convertToRelMessage);
+    MessageStream<SamzaSqlRelMessage> samzaSqlRelMessageStream = inputStream.map(new ScanMapFunction(sourceName));
 
     context.registerMessageStream(tableScan.getId(), samzaSqlRelMessageStream);
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java
index fd5195b..7a25efb 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/TranslatorContext.java
@@ -25,6 +25,7 @@ import java.util.Map;
 import org.apache.calcite.DataContext;
 import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelRoot;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
@@ -35,16 +36,25 @@ import org.apache.samza.operators.MessageStream;
 import org.apache.samza.operators.StreamGraph;
 import org.apache.samza.sql.data.RexToJavaCompiler;
 import org.apache.samza.sql.data.SamzaSqlExecutionContext;
+import org.apache.samza.sql.interfaces.SamzaRelConverter;
 
 
 /**
  * State that is maintained while translating the Calcite relational graph to Samza {@link StreamGraph}.
  */
-public class TranslatorContext {
+public class TranslatorContext implements Cloneable {
+  /**
+   * The internal variables that are shared among all cloned {@link TranslatorContext}
+   */
   private final StreamGraph streamGraph;
-  private final Map<Integer, MessageStream> messsageStreams = new HashMap<>();
   private final RexToJavaCompiler compiler;
+  private final Map<String, SamzaRelConverter> relSamzaConverters;
+  private final Map<Integer, MessageStream> messsageStreams;
+  private final Map<Integer, RelNode> relNodes;
 
+  /**
+   * The internal variables that are not shared among all cloned {@link TranslatorContext}
+   */
   private final SamzaSqlExecutionContext executionContext;
   private final DataContextImpl dataContext;
 
@@ -90,17 +100,42 @@ public class TranslatorContext {
     }
   }
 
+  private RexToJavaCompiler createExpressionCompiler(RelRoot relRoot) {
+    RelDataTypeFactory dataTypeFactory = relRoot.project().getCluster().getTypeFactory();
+    RexBuilder rexBuilder = new SamzaSqlRexBuilder(dataTypeFactory);
+    return new RexToJavaCompiler(rexBuilder);
+  }
+
+  /**
+   * Private constructor to make a clone of {@link TranslatorContext} object
+   *
+   * @param other the original object to copy from
+   */
+  private TranslatorContext(TranslatorContext other) {
+    this.streamGraph  = other.streamGraph;
+    this.compiler = other.compiler;
+    this.relSamzaConverters = other.relSamzaConverters;
+    this.messsageStreams = other.messsageStreams;
+    this.relNodes = other.relNodes;
+    this.executionContext = other.executionContext.clone();
+    this.dataContext = new DataContextImpl();
+  }
+
   /**
    * Create the instance of TranslatorContext
    * @param streamGraph Samza's streamGraph that is populated during the translation.
    * @param relRoot Root of the relational graph from calcite.
    * @param executionContext the execution context
+   * @param converters the map of schema to RelData converters
    */
-  public TranslatorContext(StreamGraph streamGraph, RelRoot relRoot, SamzaSqlExecutionContext executionContext) {
+  TranslatorContext(StreamGraph streamGraph, RelRoot relRoot, SamzaSqlExecutionContext executionContext, Map<String, SamzaRelConverter> converters) {
     this.streamGraph = streamGraph;
     this.compiler = createExpressionCompiler(relRoot);
     this.executionContext = executionContext;
     this.dataContext = new DataContextImpl();
+    this.relSamzaConverters = converters;
+    this.messsageStreams = new HashMap<>();
+    this.relNodes = new HashMap<>();
   }
 
   /**
@@ -112,22 +147,16 @@ public class TranslatorContext {
     return streamGraph;
   }
 
-  private RexToJavaCompiler createExpressionCompiler(RelRoot relRoot) {
-    RelDataTypeFactory dataTypeFactory = relRoot.project().getCluster().getTypeFactory();
-    RexBuilder rexBuilder = new SamzaSqlRexBuilder(dataTypeFactory);
-    return new RexToJavaCompiler(rexBuilder);
-  }
-
   /**
    * Gets execution context.
    *
    * @return the execution context
    */
-  public SamzaSqlExecutionContext getExecutionContext() {
+  SamzaSqlExecutionContext getExecutionContext() {
     return executionContext;
   }
 
-  public DataContext getDataContext() {
+  DataContext getDataContext() {
     return dataContext;
   }
 
@@ -136,7 +165,7 @@ public class TranslatorContext {
    *
    * @return the expression compiler
    */
-  public RexToJavaCompiler getExpressionCompiler() {
+  RexToJavaCompiler getExpressionCompiler() {
     return compiler;
   }
 
@@ -146,7 +175,7 @@ public class TranslatorContext {
    * @param id the id
    * @param stream the stream
    */
-  public void registerMessageStream(int id, MessageStream stream) {
+  void registerMessageStream(int id, MessageStream stream) {
     messsageStreams.put(id, stream);
   }
 
@@ -156,7 +185,29 @@ public class TranslatorContext {
    * @param id the id
    * @return the message stream
    */
-  public MessageStream getMessageStream(int id) {
+  MessageStream getMessageStream(int id) {
     return messsageStreams.get(id);
   }
+
+  void registerRelNode(int id, RelNode relNode) {
+    relNodes.put(id, relNode);
+  }
+
+  RelNode getRelNode(int id) {
+    return relNodes.get(id);
+  }
+
+  SamzaRelConverter getMsgConverter(String source) {
+    return this.relSamzaConverters.get(source);
+  }
+
+  /**
+   * This method helps to create a per task instance of translator context
+   *
+   * @return the cloned instance of {@link TranslatorContext}
+   */
+  @Override
+  public TranslatorContext clone() {
+    return new TranslatorContext(this);
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/test/java/org/apache/samza/sql/TestQueryTranslator.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/TestQueryTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/TestQueryTranslator.java
deleted file mode 100644
index de0ecf1..0000000
--- a/samza-sql/src/test/java/org/apache/samza/sql/TestQueryTranslator.java
+++ /dev/null
@@ -1,510 +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.sql;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.samza.SamzaException;
-import org.apache.samza.config.Config;
-import org.apache.samza.config.MapConfig;
-import org.apache.samza.operators.StreamGraphImpl;
-import org.apache.samza.operators.spec.OperatorSpec;
-import org.apache.samza.runtime.LocalApplicationRunner;
-import org.apache.samza.sql.impl.ConfigBasedIOResolverFactory;
-import org.apache.samza.sql.runner.SamzaSqlApplicationConfig;
-import org.apache.samza.sql.runner.SamzaSqlApplicationRunner;
-import org.apache.samza.sql.testutil.SamzaSqlQueryParser;
-import org.apache.samza.sql.testutil.SamzaSqlTestConfig;
-import org.apache.samza.sql.translator.QueryTranslator;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-
-public class TestQueryTranslator {
-
-  private final Map<String, String> configs = new HashMap<>();
-
-  @Before
-  public void setUp() {
-    configs.put("job.default.system", "kafka");
-  }
-
-  @Test
-  public void testTranslate() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10);
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT,
-        "Insert into testavro.outputTopic select MyTest(id) from testavro.level1.level2.SIMPLE1 as s where s.id = 10");
-    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(new LocalApplicationRunner(samzaConfig), samzaConfig);
-    translator.translate(queryInfo, streamGraph);
-    Assert.assertEquals(1, streamGraph.getOutputStreams().size());
-    Assert.assertEquals("testavro", streamGraph.getOutputStreams().keySet().stream().findFirst().get().getSystemName());
-    Assert.assertEquals("outputTopic", streamGraph.getOutputStreams().keySet().stream().findFirst().get().getPhysicalName());
-    Assert.assertEquals(1, streamGraph.getInputOperators().size());
-    Assert.assertEquals("testavro",
-        streamGraph.getInputOperators().keySet().stream().findFirst().get().getSystemName());
-    Assert.assertEquals("SIMPLE1",
-        streamGraph.getInputOperators().keySet().stream().findFirst().get().getPhysicalName());
-  }
-
-  @Test
-  public void testTranslateComplex() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10);
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT,
-        "Insert into testavro.outputTopic select Flatten(array_values) from testavro.COMPLEX1");
-//    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT,
-//        "Insert into testavro.foo2 select string_value, SUM(id) from testavro.COMPLEX1 "
-//            + "GROUP BY TumbleWindow(CURRENT_TIME, INTERVAL '1' HOUR), string_value");
-    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(new LocalApplicationRunner(samzaConfig), samzaConfig);
-    translator.translate(queryInfo, streamGraph);
-    Assert.assertEquals(1, streamGraph.getOutputStreams().size());
-    Assert.assertEquals("testavro", streamGraph.getOutputStreams().keySet().stream().findFirst().get().getSystemName());
-    Assert.assertEquals("outputTopic", streamGraph.getOutputStreams().keySet().stream().findFirst().get().getPhysicalName());
-    Assert.assertEquals(1, streamGraph.getInputOperators().size());
-    Assert.assertEquals("testavro",
-        streamGraph.getInputOperators().keySet().stream().findFirst().get().getSystemName());
-    Assert.assertEquals("COMPLEX1",
-        streamGraph.getInputOperators().keySet().stream().findFirst().get().getPhysicalName());
-  }
-
-  @Test
-  public void testTranslateSubQuery() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10);
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT,
-        "Insert into testavro.outputTopic select Flatten(a), id from (select id, array_values a, string_value s from testavro.COMPLEX1)");
-    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(new LocalApplicationRunner(samzaConfig), samzaConfig);
-    translator.translate(queryInfo, streamGraph);
-    Assert.assertEquals(1, streamGraph.getOutputStreams().size());
-    Assert.assertEquals("testavro", streamGraph.getOutputStreams().keySet().stream().findFirst().get().getSystemName());
-    Assert.assertEquals("outputTopic", streamGraph.getOutputStreams().keySet().stream().findFirst().get().getPhysicalName());
-    Assert.assertEquals(1, streamGraph.getInputOperators().size());
-    Assert.assertEquals("testavro",
-        streamGraph.getInputOperators().keySet().stream().findFirst().get().getSystemName());
-    Assert.assertEquals("COMPLEX1",
-        streamGraph.getInputOperators().keySet().stream().findFirst().get().getPhysicalName());
-  }
-
-  @Test (expected = SamzaException.class)
-  public void testTranslateStreamTableJoinWithoutJoinOperator() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
-    String sql =
-        "Insert into testavro.enrichedPageViewTopic"
-            + " select p.name as profileName, pv.pageKey"
-            + " from testavro.PAGEVIEW as pv, testavro.PROFILE.`$table` as p"
-            + " where p.id = pv.profileId";
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
-    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(new LocalApplicationRunner(samzaConfig), samzaConfig);
-    translator.translate(queryInfo, streamGraph);
-  }
-
-  @Test (expected = SamzaException.class)
-  public void testTranslateStreamTableJoinWithFullJoinOperator() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
-    String sql =
-        "Insert into testavro.enrichedPageViewTopic"
-            + " select p.name as profileName, pv.pageKey"
-            + " from testavro.PAGEVIEW as pv"
-            + " full join testavro.PROFILE.`$table` as p"
-            + " on p.id = pv.profileId";
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
-    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(new LocalApplicationRunner(samzaConfig), samzaConfig);
-    translator.translate(queryInfo, streamGraph);
-  }
-
-  @Test (expected = IllegalStateException.class)
-  public void testTranslateStreamTableJoinWithSelfJoinOperator() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
-    String sql =
-        "Insert into testavro.enrichedPageViewTopic"
-            + " select p1.name as profileName"
-            + " from testavro.PROFILE.`$table` as p1"
-            + " join testavro.PROFILE.`$table` as p2"
-            + " on p1.id = p2.id";
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
-    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(new LocalApplicationRunner(samzaConfig), samzaConfig);
-    translator.translate(queryInfo, streamGraph);
-  }
-
-  @Test (expected = SamzaException.class)
-  public void testTranslateStreamTableJoinWithThetaCondition() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
-    String sql =
-        "Insert into testavro.enrichedPageViewTopic"
-            + " select p.name as profileName, pv.pageKey"
-            + " from testavro.PAGEVIEW as pv"
-            + " join testavro.PROFILE.`$table` as p"
-            + " on p.id <> pv.profileId";
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
-    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(new LocalApplicationRunner(samzaConfig), samzaConfig);
-    translator.translate(queryInfo, streamGraph);
-  }
-
-  @Test (expected = SamzaException.class)
-  public void testTranslateStreamTableCrossJoin() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
-    String sql =
-        "Insert into testavro.enrichedPageViewTopic"
-            + " select p.name as profileName, pv.pageKey"
-            + " from testavro.PAGEVIEW as pv, testavro.PROFILE.`$table` as p";
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
-    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(new LocalApplicationRunner(samzaConfig), samzaConfig);
-    translator.translate(queryInfo, streamGraph);
-  }
-
-  @Test (expected = SamzaException.class)
-  public void testTranslateStreamTableJoinWithAndLiteralCondition() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
-    String sql =
-        "Insert into testavro.enrichedPageViewTopic"
-            + " select p.name as profileName, pv.pageKey"
-            + " from testavro.PAGEVIEW as pv"
-            + " join testavro.PROFILE.`$table` as p"
-            + " on p.id = pv.profileId and p.name = 'John'";
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
-    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(new LocalApplicationRunner(samzaConfig), samzaConfig);
-    translator.translate(queryInfo, streamGraph);
-  }
-
-  @Test (expected = SamzaException.class)
-  public void testTranslateStreamTableJoinWithSubQuery() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
-    String sql =
-        "Insert into testavro.enrichedPageViewTopic"
-            + " select p.name as profileName, pv.pageKey"
-            + " from testavro.PAGEVIEW as pv"
-            + " where exists "
-            + " (select p.id from testavro.PROFILE.`$table` as p"
-            + " where p.id = pv.profileId)";
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
-    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(new LocalApplicationRunner(samzaConfig), samzaConfig);
-    translator.translate(queryInfo, streamGraph);
-  }
-
-  @Test (expected = SamzaException.class)
-  public void testTranslateTableTableJoin() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
-    String sql =
-        "Insert into testavro.enrichedPageViewTopic"
-            + " select p.name as profileName, pv.pageKey"
-            + " from testavro.PAGEVIEW.`$table` as pv"
-            + " join testavro.PROFILE.`$table` as p"
-            + " on p.id = pv.profileId";
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
-    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(new LocalApplicationRunner(samzaConfig), samzaConfig);
-    translator.translate(queryInfo, streamGraph);
-  }
-
-  @Test (expected = SamzaException.class)
-  public void testTranslateStreamStreamJoin() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
-    String sql =
-        "Insert into testavro.enrichedPageViewTopic"
-            + " select p.name as profileName, pv.pageKey"
-            + " from testavro.PAGEVIEW as pv"
-            + " join testavro.PROFILE as p"
-            + " on p.id = pv.profileId";
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
-    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(new LocalApplicationRunner(samzaConfig), samzaConfig);
-    translator.translate(queryInfo, streamGraph);
-  }
-
-  @Test (expected = SamzaException.class)
-  public void testTranslateJoinWithIncorrectLeftJoin() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
-    String sql =
-        "Insert into testavro.enrichedPageViewTopic"
-            + " select p.name as profileName, pv.pageKey"
-            + " from testavro.PAGEVIEW.`$table` as pv"
-            + " left join testavro.PROFILE as p"
-            + " on p.id = pv.profileId";
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
-    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(new LocalApplicationRunner(samzaConfig), samzaConfig);
-    translator.translate(queryInfo, streamGraph);
-  }
-
-  @Test (expected = SamzaException.class)
-  public void testTranslateJoinWithIncorrectRightJoin() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
-    String sql =
-        "Insert into testavro.enrichedPageViewTopic"
-            + " select p.name as profileName, pv.pageKey"
-            + " from testavro.PAGEVIEW as pv"
-            + " right join testavro.PROFILE.`$table` as p"
-            + " on p.id = pv.profileId";
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
-    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(new LocalApplicationRunner(samzaConfig), samzaConfig);
-    translator.translate(queryInfo, streamGraph);
-  }
-
-  @Test (expected = SamzaException.class)
-  public void testTranslateStreamTableInnerJoinWithMissingStream() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 10);
-    String configIOResolverDomain =
-        String.format(SamzaSqlApplicationConfig.CFG_FMT_SOURCE_RESOLVER_DOMAIN, "config");
-    config.put(configIOResolverDomain + SamzaSqlApplicationConfig.CFG_FACTORY,
-        ConfigBasedIOResolverFactory.class.getName());
-    String sql =
-        "Insert into testavro.enrichedPageViewTopic"
-            + " select p.name as profileName, pv.pageKey"
-            + " from testavro.PAGEVIEW as pv"
-            + " join testavro.`$table` as p"
-            + " on p.id = pv.profileId";
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
-    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(new LocalApplicationRunner(samzaConfig), samzaConfig);
-    translator.translate(queryInfo, streamGraph);
-  }
-
-  @Test (expected = SamzaException.class)
-  public void testTranslateStreamTableInnerJoinWithUdf() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 10);
-    String sql =
-        "Insert into testavro.enrichedPageViewTopic"
-            + " select p.name as profileName, pv.pageKey"
-            + " from testavro.PAGEVIEW as pv"
-            + " join testavro.PROFILE.`$table` as p"
-            + " on MyTest(p.id) = MyTest(pv.profileId)";
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
-    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(new LocalApplicationRunner(samzaConfig), samzaConfig);
-    translator.translate(queryInfo, streamGraph);
-  }
-
-  @Test
-  public void testTranslateStreamTableInnerJoin() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 10);
-    String sql =
-        "Insert into testavro.enrichedPageViewTopic"
-            + " select p.name as profileName, pv.pageKey"
-            + " from testavro.PAGEVIEW as pv"
-            + " join testavro.PROFILE.`$table` as p"
-            + " on p.id = pv.profileId";
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
-    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(new LocalApplicationRunner(samzaConfig), samzaConfig);
-    translator.translate(queryInfo, streamGraph);
-
-    Assert.assertEquals(2, streamGraph.getOutputStreams().size());
-    Assert.assertEquals("kafka", streamGraph.getOutputStreams().keySet().stream().findFirst().get().getSystemName());
-    Assert.assertEquals("sql-job-1-partition_by-stream_1", streamGraph.getOutputStreams().keySet().stream().findFirst().get().getPhysicalName());
-    Assert.assertEquals("testavro", streamGraph.getOutputStreams().keySet().stream().skip(1).findFirst().get().getSystemName());
-    Assert.assertEquals("enrichedPageViewTopic", streamGraph.getOutputStreams().keySet().stream().skip(1).findFirst().get().getPhysicalName());
-
-    Assert.assertEquals(3, streamGraph.getInputOperators().size());
-    Assert.assertEquals("testavro",
-        streamGraph.getInputOperators().keySet().stream().findFirst().get().getSystemName());
-    Assert.assertEquals("PAGEVIEW",
-        streamGraph.getInputOperators().keySet().stream().findFirst().get().getPhysicalName());
-    Assert.assertEquals("testavro",
-        streamGraph.getInputOperators().keySet().stream().skip(1).findFirst().get().getSystemName());
-    Assert.assertEquals("PROFILE",
-        streamGraph.getInputOperators().keySet().stream().skip(1).findFirst().get().getPhysicalName());
-    Assert.assertEquals("kafka",
-        streamGraph.getInputOperators().keySet().stream().skip(2).findFirst().get().getSystemName());
-    Assert.assertEquals("sql-job-1-partition_by-stream_1",
-        streamGraph.getInputOperators().keySet().stream().skip(2).findFirst().get().getPhysicalName());
-  }
-
-  @Test
-  public void testTranslateStreamTableLeftJoin() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 10);
-    String sql =
-        "Insert into testavro.enrichedPageViewTopic"
-            + " select p.name as profileName, pv.pageKey"
-            + " from testavro.PAGEVIEW as pv"
-            + " left join testavro.PROFILE.`$table` as p"
-            + " on p.id = pv.profileId";
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
-    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(new LocalApplicationRunner(samzaConfig), samzaConfig);
-    translator.translate(queryInfo, streamGraph);
-
-    Assert.assertEquals(2, streamGraph.getOutputStreams().size());
-    Assert.assertEquals("kafka", streamGraph.getOutputStreams().keySet().stream().findFirst().get().getSystemName());
-    Assert.assertEquals("sql-job-1-partition_by-stream_1",
-        streamGraph.getOutputStreams().keySet().stream().findFirst().get().getPhysicalName());
-    Assert.assertEquals("testavro", streamGraph.getOutputStreams().keySet().stream().skip(1).findFirst().get().getSystemName());
-    Assert.assertEquals("enrichedPageViewTopic",
-        streamGraph.getOutputStreams().keySet().stream().skip(1).findFirst().get().getPhysicalName());
-
-    Assert.assertEquals(3, streamGraph.getInputOperators().size());
-    Assert.assertEquals("testavro",
-        streamGraph.getInputOperators().keySet().stream().findFirst().get().getSystemName());
-    Assert.assertEquals("PAGEVIEW",
-        streamGraph.getInputOperators().keySet().stream().findFirst().get().getPhysicalName());
-    Assert.assertEquals("testavro",
-        streamGraph.getInputOperators().keySet().stream().skip(1).findFirst().get().getSystemName());
-    Assert.assertEquals("PROFILE",
-        streamGraph.getInputOperators().keySet().stream().skip(1).findFirst().get().getPhysicalName());
-    Assert.assertEquals("kafka",
-        streamGraph.getInputOperators().keySet().stream().skip(2).findFirst().get().getSystemName());
-    Assert.assertEquals("sql-job-1-partition_by-stream_1",
-        streamGraph.getInputOperators().keySet().stream().skip(2).findFirst().get().getPhysicalName());
-  }
-
-  @Test
-  public void testTranslateStreamTableRightJoin() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 10);
-    String sql =
-        "Insert into testavro.enrichedPageViewTopic"
-            + " select p.name as profileName, pv.pageKey"
-            + " from testavro.PROFILE.`$table` as p"
-            + " right join testavro.PAGEVIEW as pv"
-            + " on p.id = pv.profileId";
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
-    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(new LocalApplicationRunner(samzaConfig), samzaConfig);
-    translator.translate(queryInfo, streamGraph);
-
-    Assert.assertEquals(2, streamGraph.getOutputStreams().size());
-    Assert.assertEquals("kafka", streamGraph.getOutputStreams().keySet().stream().findFirst().get().getSystemName());
-    Assert.assertEquals("sql-job-1-partition_by-stream_1",
-        streamGraph.getOutputStreams().keySet().stream().findFirst().get().getPhysicalName());
-    Assert.assertEquals("testavro", streamGraph.getOutputStreams().keySet().stream().skip(1).findFirst().get().getSystemName());
-    Assert.assertEquals("enrichedPageViewTopic",
-        streamGraph.getOutputStreams().keySet().stream().skip(1).findFirst().get().getPhysicalName());
-
-    Assert.assertEquals(3, streamGraph.getInputOperators().size());
-    Assert.assertEquals("testavro",
-        streamGraph.getInputOperators().keySet().stream().findFirst().get().getSystemName());
-    Assert.assertEquals("PROFILE",
-        streamGraph.getInputOperators().keySet().stream().findFirst().get().getPhysicalName());
-    Assert.assertEquals("testavro",
-        streamGraph.getInputOperators().keySet().stream().skip(1).findFirst().get().getSystemName());
-    Assert.assertEquals("PAGEVIEW",
-        streamGraph.getInputOperators().keySet().stream().skip(1).findFirst().get().getPhysicalName());
-    Assert.assertEquals("kafka",
-        streamGraph.getInputOperators().keySet().stream().skip(2).findFirst().get().getSystemName());
-    Assert.assertEquals("sql-job-1-partition_by-stream_1",
-        streamGraph.getInputOperators().keySet().stream().skip(2).findFirst().get().getPhysicalName());
-  }
-
-  @Test
-  public void testTranslateGroupBy() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 10);
-    String sql =
-        "Insert into testavro.pageViewCountTopic"
-            + " select 'SampleJob' as jobName, pv.pageKey, count(*) as `count`"
-            + " from testavro.PAGEVIEW as pv"
-            + " where pv.pageKey = 'job' or pv.pageKey = 'inbox'"
-            + " group by (pv.pageKey)";
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
-    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(new LocalApplicationRunner(samzaConfig), samzaConfig);
-    translator.translate(queryInfo, streamGraph);
-
-    Assert.assertEquals(1, streamGraph.getInputOperators().size());
-    Assert.assertEquals(1, streamGraph.getOutputStreams().size());
-    Assert.assertTrue(streamGraph.hasWindowOrJoins());
-    Collection<OperatorSpec> operatorSpecs = streamGraph.getAllOperatorSpecs();
-  }
-
-  @Test (expected = SamzaException.class)
-  public void testTranslateGroupByWithSumAggregator() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 10);
-    String sql =
-        "Insert into testavro.pageViewCountTopic"
-            + " select 'SampleJob' as jobName, pv.pageKey, sum(pv.profileId) as `sum`"
-            + " from testavro.PAGEVIEW as pv" + " where pv.pageKey = 'job' or pv.pageKey = 'inbox'"
-            + " group by (pv.pageKey)";
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
-    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(new LocalApplicationRunner(samzaConfig), samzaConfig);
-    translator.translate(queryInfo, streamGraph);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlApplicationConfig.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlApplicationConfig.java b/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlApplicationConfig.java
deleted file mode 100644
index 0d48c56..0000000
--- a/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlApplicationConfig.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*   http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing,
-* software distributed under the License is distributed on an
-* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-* KIND, either express or implied.  See the License for the
-* specific language governing permissions and limitations
-* under the License.
-*/
-
-package org.apache.samza.sql;
-
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.samza.SamzaException;
-import org.apache.samza.config.MapConfig;
-import org.apache.samza.sql.impl.ConfigBasedUdfResolver;
-import org.apache.samza.sql.interfaces.SqlIOConfig;
-import org.apache.samza.sql.runner.SamzaSqlApplicationConfig;
-import org.apache.samza.sql.testutil.SamzaSqlTestConfig;
-import org.junit.Assert;
-import org.junit.Test;
-
-
-public class TestSamzaSqlApplicationConfig {
-
-  @Test
-  public void testConfigInit() {
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10);
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, "Insert into testavro.COMPLEX1 select * from testavro.SIMPLE1");
-    String configUdfResolverDomain = String.format(SamzaSqlApplicationConfig.CFG_FMT_UDF_RESOLVER_DOMAIN, "config");
-    int numUdfs = config.get(configUdfResolverDomain + ConfigBasedUdfResolver.CFG_UDF_CLASSES).split(",").length;
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    Assert.assertEquals(1, samzaSqlApplicationConfig.getQueryInfo().size());
-    Assert.assertEquals(numUdfs, samzaSqlApplicationConfig.getUdfMetadata().size());
-    Assert.assertEquals(1, samzaSqlApplicationConfig.getInputSystemStreamConfigBySource().size());
-    Assert.assertEquals(1, samzaSqlApplicationConfig.getOutputSystemStreamConfigsBySource().size());
-  }
-
-  @Test
-  public void testWrongConfigs() {
-
-    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10);
-
-
-    try {
-      // Fail because no SQL config
-      new SamzaSqlApplicationConfig(new MapConfig(config));
-      Assert.fail();
-    } catch (SamzaException e) {
-    }
-
-    // Pass
-    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, "Insert into testavro.COMPLEX1 select * from testavro.SIMPLE1");
-    new SamzaSqlApplicationConfig(new MapConfig(config));
-    testWithoutConfigShouldFail(config, SamzaSqlApplicationConfig.CFG_IO_RESOLVER);
-    testWithoutConfigShouldFail(config, SamzaSqlApplicationConfig.CFG_UDF_RESOLVER);
-
-    String configIOResolverDomain =
-        String.format(SamzaSqlApplicationConfig.CFG_FMT_SOURCE_RESOLVER_DOMAIN, "config");
-    String avroSamzaSqlConfigPrefix = configIOResolverDomain + String.format("%s.", "testavro");
-
-    testWithoutConfigShouldFail(config, avroSamzaSqlConfigPrefix + SqlIOConfig.CFG_SAMZA_REL_CONVERTER);
-
-    // Configs for the unused system "log" is not mandatory.
-    String logSamzaSqlConfigPrefix = configIOResolverDomain + String.format("%s.", "log");
-    testWithoutConfigShouldPass(config, logSamzaSqlConfigPrefix + SqlIOConfig.CFG_SAMZA_REL_CONVERTER);
-  }
-
-  private void testWithoutConfigShouldPass(Map<String, String> config, String configKey) {
-    Map<String, String> badConfigs = new HashMap<>(config);
-    badConfigs.remove(configKey);
-    new SamzaSqlApplicationConfig(new MapConfig(badConfigs));
-  }
-
-  private void testWithoutConfigShouldFail(Map<String, String> config, String configKey) {
-    Map<String, String> badConfigs = new HashMap<>(config);
-    badConfigs.remove(configKey);
-    try {
-      new SamzaSqlApplicationConfig(new MapConfig(badConfigs));
-      Assert.fail();
-    } catch (IllegalArgumentException e) {
-      // swallow
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlApplicationRunner.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlApplicationRunner.java b/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlApplicationRunner.java
deleted file mode 100644
index e42b55d..0000000
--- a/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlApplicationRunner.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*   http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing,
-* software distributed under the License is distributed on an
-* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-* KIND, either express or implied.  See the License for the
-* specific language governing permissions and limitations
-* under the License.
-*/
-
-package org.apache.samza.sql;
-
-import java.util.Map;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.config.MapConfig;
-import org.apache.samza.runtime.LocalApplicationRunner;
-import org.apache.samza.runtime.RemoteApplicationRunner;
-import org.apache.samza.sql.runner.SamzaSqlApplicationRunner;
-import org.apache.samza.sql.testutil.SamzaSqlTestConfig;
-import org.junit.Assert;
-
-import org.apache.samza.sql.runner.SamzaSqlApplicationConfig;
-import org.junit.Test;
-
-
-public class TestSamzaSqlApplicationRunner {
-
-  @Test
-  public void testComputeSamzaConfigs() {
-    Map<String, String> configs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10);
-    String sql1 = "Insert into testavro.outputTopic select id, MyTest(id) as long_value from testavro.SIMPLE1";
-    configs.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql1);
-    configs.put(SamzaSqlApplicationRunner.RUNNER_CONFIG, SamzaSqlApplicationRunner.class.getName());
-    MapConfig samzaConfig = new MapConfig(configs);
-    Config newConfigs = SamzaSqlApplicationRunner.computeSamzaConfigs(true, samzaConfig);
-    Assert.assertEquals(newConfigs.get(SamzaSqlApplicationRunner.RUNNER_CONFIG), LocalApplicationRunner.class.getName());
-    // Check whether three new configs added.
-    Assert.assertEquals(newConfigs.size(), configs.size() + 3);
-
-    newConfigs = SamzaSqlApplicationRunner.computeSamzaConfigs(false, samzaConfig);
-    Assert.assertEquals(newConfigs.get(SamzaSqlApplicationRunner.RUNNER_CONFIG), RemoteApplicationRunner.class.getName());
-
-    // Check whether three new configs added.
-    Assert.assertEquals(newConfigs.size(), configs.size() + 3);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlFileParser.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlFileParser.java b/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlFileParser.java
deleted file mode 100644
index 5bac472..0000000
--- a/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlFileParser.java
+++ /dev/null
@@ -1,58 +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.sql;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.util.List;
-
-import org.apache.samza.sql.testutil.SqlFileParser;
-import org.junit.Assert;
-import org.junit.Test;
-
-
-public class TestSamzaSqlFileParser {
-
-  public static final String TEST_SQL =
-      "insert into log.outputStream \n" + "\tselect * from brooklin.elasticsearchEnterpriseAccounts\n"
-          + "insert into log.outputstream select sfdcAccountId as key, organizationUrn as name2, "
-          + "description as name3 from brooklin.elasticsearchEnterpriseAccounts\n" + "--insert into log.outputstream \n"
-          + "insert into log.outputstream \n" + "\n" + "\tselect id, MyTest(id) as id2 \n" + "\n"
-          + "\tfrom tracking.SamzaSqlTestTopic1_p8";
-
-  @Test
-  public void testParseSqlFile() throws IOException {
-    File tempFile = File.createTempFile("testparser", "");
-    PrintWriter fileWriter = new PrintWriter(tempFile.getCanonicalPath());
-    fileWriter.println(TEST_SQL);
-    fileWriter.close();
-
-    List<String> sqlStmts = SqlFileParser.parseSqlFile(tempFile.getAbsolutePath());
-    Assert.assertEquals(3, sqlStmts.size());
-    Assert.assertEquals("insert into log.outputStream select * from brooklin.elasticsearchEnterpriseAccounts",
-        sqlStmts.get(0));
-    Assert.assertEquals(
-        "insert into log.outputstream select sfdcAccountId as key, organizationUrn as name2, description as name3 from brooklin.elasticsearchEnterpriseAccounts",
-        sqlStmts.get(1));
-    Assert.assertEquals("insert into log.outputstream select id, MyTest(id) as id2 from tracking.SamzaSqlTestTopic1_p8",
-        sqlStmts.get(2));
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlQueryParser.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlQueryParser.java b/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlQueryParser.java
deleted file mode 100644
index 24faf4b..0000000
--- a/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlQueryParser.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*   http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing,
-* software distributed under the License is distributed on an
-* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-* KIND, either express or implied.  See the License for the
-* specific language governing permissions and limitations
-* under the License.
-*/
-
-package org.apache.samza.sql;
-
-import org.apache.samza.SamzaException;
-import org.apache.samza.sql.testutil.SamzaSqlQueryParser;
-import org.apache.samza.sql.testutil.SamzaSqlQueryParser.QueryInfo;
-import org.junit.Test;
-
-import junit.framework.Assert;
-
-public class TestSamzaSqlQueryParser {
-
-  @Test
-  public void testParseQuery() {
-    QueryInfo queryInfo = SamzaSqlQueryParser.parseQuery("insert into log.foo select * from tracking.bar");
-    Assert.assertEquals("log.foo", queryInfo.getSink());
-    Assert.assertEquals(queryInfo.getSelectQuery(), "select * from tracking.bar", queryInfo.getSelectQuery());
-    Assert.assertEquals(1, queryInfo.getSources().size());
-    Assert.assertEquals("tracking.bar", queryInfo.getSources().get(0));
-  }
-
-  @Test
-  public void testParseJoinQuery() {
-    String sql =
-        "Insert into testavro.enrichedPageViewTopic"
-            + " select p.name as profileName, pv.pageKey"
-            + " from testavro.PAGEVIEW as pv"
-            + " join testavro.PROFILE.`$table` as p"
-            + " on p.id = pv.profileId";
-    QueryInfo queryInfo = SamzaSqlQueryParser.parseQuery(sql);
-    Assert.assertEquals("testavro.enrichedPageViewTopic", queryInfo.getSink());
-    Assert.assertEquals(2, queryInfo.getSources().size());
-    Assert.assertEquals("testavro.PAGEVIEW", queryInfo.getSources().get(0));
-    Assert.assertEquals("testavro.PROFILE.$table", queryInfo.getSources().get(1));
-  }
-
-  @Test
-  public void testParseInvalidQuery() {
-
-    try {
-      SamzaSqlQueryParser.parseQuery("select * from tracking.bar");
-      Assert.fail("Expected a samzaException");
-    } catch (SamzaException e) {
-    }
-
-    try {
-      SamzaSqlQueryParser.parseQuery("insert into select * from tracking.bar");
-      Assert.fail("Expected a samzaException");
-    } catch (SamzaException e) {
-    }
-
-    try {
-      SamzaSqlQueryParser.parseQuery("insert into log.off select from tracking.bar");
-      Assert.fail("Expected a samzaException");
-    } catch (SamzaException e) {
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlRelMessage.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlRelMessage.java b/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlRelMessage.java
deleted file mode 100644
index 689af72..0000000
--- a/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlRelMessage.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.sql;
-
-import java.util.Arrays;
-import java.util.List;
-import org.apache.samza.sql.data.SamzaSqlRelMessage;
-import org.junit.Assert;
-import org.junit.Test;
-
-
-public class TestSamzaSqlRelMessage {
-
-  private List<Object> values = Arrays.asList("value1", "value2");
-  private List<String> names = Arrays.asList("field1", "field2");
-
-  @Test
-  public void testGetField() {
-    SamzaSqlRelMessage message = new SamzaSqlRelMessage(names, values);
-    Assert.assertEquals(values.get(0), message.getSamzaSqlRelRecord().getField(names.get(0)).get());
-    Assert.assertEquals(values.get(1), message.getSamzaSqlRelRecord().getField(names.get(1)).get());
-  }
-
-  @Test
-  public void testGetNonExistentField() {
-    SamzaSqlRelMessage message = new SamzaSqlRelMessage(names, values);
-    Assert.assertFalse(message.getSamzaSqlRelRecord().getField("field3").isPresent());
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlRelMessageJoinFunction.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlRelMessageJoinFunction.java b/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlRelMessageJoinFunction.java
deleted file mode 100644
index 90fce3b..0000000
--- a/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlRelMessageJoinFunction.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*   http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing,
-* software distributed under the License is distributed on an
-* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-* KIND, either express or implied.  See the License for the
-* specific language governing permissions and limitations
-* under the License.
-*/
-
-package org.apache.samza.sql;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.stream.Collectors;
-import org.apache.calcite.rel.core.JoinRelType;
-import org.apache.samza.operators.KV;
-import org.apache.samza.sql.data.SamzaSqlCompositeKey;
-import org.apache.samza.sql.data.SamzaSqlRelMessage;
-import org.apache.samza.sql.translator.SamzaSqlRelMessageJoinFunction;
-import org.junit.Assert;
-import org.junit.Test;
-
-
-public class TestSamzaSqlRelMessageJoinFunction {
-
-  private List<String> streamFieldNames = Arrays.asList("field1", "field2", "field3", "field4");
-  private List<Object> streamFieldValues = Arrays.asList("value1", 1, null, "value4");
-  private List<String> tableFieldNames = Arrays.asList("field11", "field12", "field13", "field14");
-  private List<Object> tableFieldValues = Arrays.asList("value1", 1, null, "value5");
-
-  @Test
-  public void testWithInnerJoinWithTableOnRight() {
-    SamzaSqlRelMessage streamMsg = new SamzaSqlRelMessage(streamFieldNames, streamFieldValues);
-    SamzaSqlRelMessage tableMsg = new SamzaSqlRelMessage(tableFieldNames, tableFieldValues);
-    JoinRelType joinRelType = JoinRelType.INNER;
-    List<Integer> streamKeyIds = Arrays.asList(0, 1);
-    List<Integer> tableKeyIds = Arrays.asList(0, 1);
-    SamzaSqlCompositeKey compositeKey = SamzaSqlCompositeKey.createSamzaSqlCompositeKey(tableMsg, tableKeyIds);
-    KV<SamzaSqlCompositeKey, SamzaSqlRelMessage> record = KV.of(compositeKey, tableMsg);
-
-    SamzaSqlRelMessageJoinFunction joinFn =
-        new SamzaSqlRelMessageJoinFunction(joinRelType, true, streamKeyIds, streamFieldNames, tableFieldNames);
-    SamzaSqlRelMessage outMsg = joinFn.apply(streamMsg, record);
-
-    Assert.assertEquals(outMsg.getSamzaSqlRelRecord().getFieldValues().size(),
-        outMsg.getSamzaSqlRelRecord().getFieldNames().size());
-    List<String> expectedFieldNames = new ArrayList<>(streamFieldNames);
-    expectedFieldNames.addAll(tableFieldNames);
-    List<Object> expectedFieldValues = new ArrayList<>(streamFieldValues);
-    expectedFieldValues.addAll(tableFieldValues);
-    Assert.assertEquals(outMsg.getSamzaSqlRelRecord().getFieldValues(), expectedFieldValues);
-  }
-
-  @Test
-  public void testWithInnerJoinWithTableOnLeft() {
-    SamzaSqlRelMessage streamMsg = new SamzaSqlRelMessage(streamFieldNames, streamFieldValues);
-    SamzaSqlRelMessage tableMsg = new SamzaSqlRelMessage(tableFieldNames, tableFieldValues);
-    JoinRelType joinRelType = JoinRelType.INNER;
-    List<Integer> streamKeyIds = Arrays.asList(0, 2);
-    List<Integer> tableKeyIds = Arrays.asList(0, 2);
-    SamzaSqlCompositeKey compositeKey = SamzaSqlCompositeKey.createSamzaSqlCompositeKey(tableMsg, tableKeyIds);
-    KV<SamzaSqlCompositeKey, SamzaSqlRelMessage> record = KV.of(compositeKey, tableMsg);
-
-    SamzaSqlRelMessageJoinFunction joinFn =
-        new SamzaSqlRelMessageJoinFunction(joinRelType, false, streamKeyIds, streamFieldNames, tableFieldNames);
-    SamzaSqlRelMessage outMsg = joinFn.apply(streamMsg, record);
-
-    Assert.assertEquals(outMsg.getSamzaSqlRelRecord().getFieldValues().size(),
-        outMsg.getSamzaSqlRelRecord().getFieldNames().size());
-    List<String> expectedFieldNames = new ArrayList<>(tableFieldNames);
-    expectedFieldNames.addAll(streamFieldNames);
-    List<Object> expectedFieldValues = new ArrayList<>(tableFieldValues);
-    expectedFieldValues.addAll(streamFieldValues);
-    Assert.assertEquals(outMsg.getSamzaSqlRelRecord().getFieldValues(), expectedFieldValues);
-  }
-
-  @Test
-  public void testNullRecordWithInnerJoin() {
-    SamzaSqlRelMessage streamMsg = new SamzaSqlRelMessage(streamFieldNames, streamFieldValues);
-    JoinRelType joinRelType = JoinRelType.INNER;
-    List<Integer> streamKeyIds = Arrays.asList(0, 1);
-
-    SamzaSqlRelMessageJoinFunction joinFn =
-        new SamzaSqlRelMessageJoinFunction(joinRelType, true, streamKeyIds, streamFieldNames, tableFieldNames);
-    SamzaSqlRelMessage outMsg = joinFn.apply(streamMsg, null);
-    Assert.assertNull(outMsg);
-  }
-
-  @Test
-  public void testNullRecordWithLeftOuterJoin() {
-    SamzaSqlRelMessage streamMsg = new SamzaSqlRelMessage(streamFieldNames, streamFieldValues);
-    JoinRelType joinRelType = JoinRelType.LEFT;
-    List<Integer> streamKeyIds = Arrays.asList(0, 1);
-
-    SamzaSqlRelMessageJoinFunction joinFn =
-        new SamzaSqlRelMessageJoinFunction(joinRelType, true, streamKeyIds, streamFieldNames,
-            tableFieldNames);
-    SamzaSqlRelMessage outMsg = joinFn.apply(streamMsg, null);
-
-    Assert.assertEquals(outMsg.getSamzaSqlRelRecord().getFieldValues().size(),
-        outMsg.getSamzaSqlRelRecord().getFieldNames().size());
-    List<String> expectedFieldNames = new ArrayList<>(streamFieldNames);
-    expectedFieldNames.addAll(tableFieldNames);
-    List<Object> expectedFieldValues = new ArrayList<>(streamFieldValues);
-    expectedFieldValues.addAll(tableFieldNames.stream().map( name -> null ).collect(Collectors.toList()));
-    Assert.assertEquals(outMsg.getSamzaSqlRelRecord().getFieldValues(), expectedFieldValues);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/test/java/org/apache/samza/sql/data/TestSamzaSqlRelMessage.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/data/TestSamzaSqlRelMessage.java b/samza-sql/src/test/java/org/apache/samza/sql/data/TestSamzaSqlRelMessage.java
new file mode 100644
index 0000000..93e6223
--- /dev/null
+++ b/samza-sql/src/test/java/org/apache/samza/sql/data/TestSamzaSqlRelMessage.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.sql.data;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.samza.sql.data.SamzaSqlRelMessage;
+import org.junit.Assert;
+import org.junit.Test;
+
+
+public class TestSamzaSqlRelMessage {
+
+  private List<Object> values = Arrays.asList("value1", "value2");
+  private List<String> names = Arrays.asList("field1", "field2");
+
+  @Test
+  public void testGetField() {
+    SamzaSqlRelMessage message = new SamzaSqlRelMessage(names, values);
+    Assert.assertEquals(values.get(0), message.getSamzaSqlRelRecord().getField(names.get(0)).get());
+    Assert.assertEquals(values.get(1), message.getSamzaSqlRelRecord().getField(names.get(1)).get());
+  }
+
+  @Test
+  public void testGetNonExistentField() {
+    SamzaSqlRelMessage message = new SamzaSqlRelMessage(names, values);
+    Assert.assertFalse(message.getSamzaSqlRelRecord().getField("field3").isPresent());
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationConfig.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationConfig.java b/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationConfig.java
new file mode 100644
index 0000000..dac5d02
--- /dev/null
+++ b/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationConfig.java
@@ -0,0 +1,95 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*   http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.sql.runner;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.sql.impl.ConfigBasedUdfResolver;
+import org.apache.samza.sql.interfaces.SqlIOConfig;
+import org.apache.samza.sql.runner.SamzaSqlApplicationConfig;
+import org.apache.samza.sql.testutil.SamzaSqlTestConfig;
+import org.junit.Assert;
+import org.junit.Test;
+
+
+public class TestSamzaSqlApplicationConfig {
+
+  @Test
+  public void testConfigInit() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10);
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, "Insert into testavro.COMPLEX1 select * from testavro.SIMPLE1");
+    String configUdfResolverDomain = String.format(SamzaSqlApplicationConfig.CFG_FMT_UDF_RESOLVER_DOMAIN, "config");
+    int numUdfs = config.get(configUdfResolverDomain + ConfigBasedUdfResolver.CFG_UDF_CLASSES).split(",").length;
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    Assert.assertEquals(1, samzaSqlApplicationConfig.getQueryInfo().size());
+    Assert.assertEquals(numUdfs, samzaSqlApplicationConfig.getUdfMetadata().size());
+    Assert.assertEquals(1, samzaSqlApplicationConfig.getInputSystemStreamConfigBySource().size());
+    Assert.assertEquals(1, samzaSqlApplicationConfig.getOutputSystemStreamConfigsBySource().size());
+  }
+
+  @Test
+  public void testWrongConfigs() {
+
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10);
+
+
+    try {
+      // Fail because no SQL config
+      new SamzaSqlApplicationConfig(new MapConfig(config));
+      Assert.fail();
+    } catch (SamzaException e) {
+    }
+
+    // Pass
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, "Insert into testavro.COMPLEX1 select * from testavro.SIMPLE1");
+    new SamzaSqlApplicationConfig(new MapConfig(config));
+    testWithoutConfigShouldFail(config, SamzaSqlApplicationConfig.CFG_IO_RESOLVER);
+    testWithoutConfigShouldFail(config, SamzaSqlApplicationConfig.CFG_UDF_RESOLVER);
+
+    String configIOResolverDomain =
+        String.format(SamzaSqlApplicationConfig.CFG_FMT_SOURCE_RESOLVER_DOMAIN, "config");
+    String avroSamzaSqlConfigPrefix = configIOResolverDomain + String.format("%s.", "testavro");
+
+    testWithoutConfigShouldFail(config, avroSamzaSqlConfigPrefix + SqlIOConfig.CFG_SAMZA_REL_CONVERTER);
+
+    // Configs for the unused system "log" is not mandatory.
+    String logSamzaSqlConfigPrefix = configIOResolverDomain + String.format("%s.", "log");
+    testWithoutConfigShouldPass(config, logSamzaSqlConfigPrefix + SqlIOConfig.CFG_SAMZA_REL_CONVERTER);
+  }
+
+  private void testWithoutConfigShouldPass(Map<String, String> config, String configKey) {
+    Map<String, String> badConfigs = new HashMap<>(config);
+    badConfigs.remove(configKey);
+    new SamzaSqlApplicationConfig(new MapConfig(badConfigs));
+  }
+
+  private void testWithoutConfigShouldFail(Map<String, String> config, String configKey) {
+    Map<String, String> badConfigs = new HashMap<>(config);
+    badConfigs.remove(configKey);
+    try {
+      new SamzaSqlApplicationConfig(new MapConfig(badConfigs));
+      Assert.fail();
+    } catch (IllegalArgumentException e) {
+      // swallow
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRunner.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRunner.java b/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRunner.java
new file mode 100644
index 0000000..b6dcac5
--- /dev/null
+++ b/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRunner.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.sql.runner;
+
+import java.util.Map;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.runtime.LocalApplicationRunner;
+import org.apache.samza.runtime.RemoteApplicationRunner;
+import org.apache.samza.sql.runner.SamzaSqlApplicationRunner;
+import org.apache.samza.sql.testutil.SamzaSqlTestConfig;
+import org.junit.Assert;
+
+import org.apache.samza.sql.runner.SamzaSqlApplicationConfig;
+import org.junit.Test;
+
+
+public class TestSamzaSqlApplicationRunner {
+
+  @Test
+  public void testComputeSamzaConfigs() {
+    Map<String, String> configs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10);
+    String sql1 = "Insert into testavro.outputTopic select id, MyTest(id) as long_value from testavro.SIMPLE1";
+    configs.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql1);
+    configs.put(SamzaSqlApplicationRunner.RUNNER_CONFIG, SamzaSqlApplicationRunner.class.getName());
+    MapConfig samzaConfig = new MapConfig(configs);
+    Config newConfigs = SamzaSqlApplicationRunner.computeSamzaConfigs(true, samzaConfig);
+    Assert.assertEquals(newConfigs.get(SamzaSqlApplicationRunner.RUNNER_CONFIG), LocalApplicationRunner.class.getName());
+    // Check whether three new configs added.
+    Assert.assertEquals(newConfigs.size(), configs.size() + 3);
+
+    newConfigs = SamzaSqlApplicationRunner.computeSamzaConfigs(false, samzaConfig);
+    Assert.assertEquals(newConfigs.get(SamzaSqlApplicationRunner.RUNNER_CONFIG), RemoteApplicationRunner.class.getName());
+
+    // Check whether three new configs added.
+    Assert.assertEquals(newConfigs.size(), configs.size() + 3);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestSamzaSqlFileParser.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestSamzaSqlFileParser.java b/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestSamzaSqlFileParser.java
new file mode 100644
index 0000000..a84f347
--- /dev/null
+++ b/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestSamzaSqlFileParser.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.sql.testutil;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.List;
+
+import org.apache.samza.sql.testutil.SqlFileParser;
+import org.junit.Assert;
+import org.junit.Test;
+
+
+public class TestSamzaSqlFileParser {
+
+  public static final String TEST_SQL =
+      "insert into log.outputStream \n" + "\tselect * from brooklin.elasticsearchEnterpriseAccounts\n"
+          + "insert into log.outputstream select sfdcAccountId as key, organizationUrn as name2, "
+          + "description as name3 from brooklin.elasticsearchEnterpriseAccounts\n" + "--insert into log.outputstream \n"
+          + "insert into log.outputstream \n" + "\n" + "\tselect id, MyTest(id) as id2 \n" + "\n"
+          + "\tfrom tracking.SamzaSqlTestTopic1_p8";
+
+  @Test
+  public void testParseSqlFile() throws IOException {
+    File tempFile = File.createTempFile("testparser", "");
+    PrintWriter fileWriter = new PrintWriter(tempFile.getCanonicalPath());
+    fileWriter.println(TEST_SQL);
+    fileWriter.close();
+
+    List<String> sqlStmts = SqlFileParser.parseSqlFile(tempFile.getAbsolutePath());
+    Assert.assertEquals(3, sqlStmts.size());
+    Assert.assertEquals("insert into log.outputStream select * from brooklin.elasticsearchEnterpriseAccounts",
+        sqlStmts.get(0));
+    Assert.assertEquals(
+        "insert into log.outputstream select sfdcAccountId as key, organizationUrn as name2, description as name3 from brooklin.elasticsearchEnterpriseAccounts",
+        sqlStmts.get(1));
+    Assert.assertEquals("insert into log.outputstream select id, MyTest(id) as id2 from tracking.SamzaSqlTestTopic1_p8",
+        sqlStmts.get(2));
+  }
+}


[05/10] samza git commit: SAMZA-1659: Serializable OperatorSpec

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/operators/spec/OperatorSpecTestUtils.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/spec/OperatorSpecTestUtils.java b/samza-core/src/test/java/org/apache/samza/operators/spec/OperatorSpecTestUtils.java
new file mode 100644
index 0000000..b39b0d0
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/operators/spec/OperatorSpecTestUtils.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.samza.operators.OperatorSpecGraph;
+import org.apache.samza.operators.TableImpl;
+import org.apache.samza.operators.functions.TimerFunction;
+import org.apache.samza.operators.functions.WatermarkFunction;
+import org.apache.samza.serializers.SerializableSerde;
+import org.apache.samza.system.StreamSpec;
+import org.apache.samza.table.TableSpec;
+
+import static org.junit.Assert.*;
+
+
+/**
+ * Test util methods for {@link OperatorSpec} classes
+ */
+public class OperatorSpecTestUtils {
+  private final static SerializableSerde<OperatorSpec> SPEC_SERDE = new SerializableSerde<>();
+
+  static OperatorSpec copyOpSpec(OperatorSpec original) {
+    return SPEC_SERDE.fromBytes(SPEC_SERDE.toBytes(original));
+  }
+
+  enum TestEnum {
+    One, Two, Three
+  }
+
+  public static void assertClonedGraph(OperatorSpecGraph originalGraph, OperatorSpecGraph clonedGraph) {
+    assertClonedInputs(originalGraph.getInputOperators(), clonedGraph.getInputOperators());
+    assertClonedOutputs(originalGraph.getOutputStreams(), clonedGraph.getOutputStreams());
+    assertClonedTables(originalGraph.getTables(), clonedGraph.getTables());
+    assertAllOperators(originalGraph.getAllOperatorSpecs(), clonedGraph.getAllOperatorSpecs());
+  }
+
+  private static void assertAllOperators(Collection<OperatorSpec> originalOpSpecs, Collection<OperatorSpec> clonedOpSpecs) {
+    assertEquals(originalOpSpecs.size(), clonedOpSpecs.size());
+    List<OperatorSpec> originalList = new ArrayList<>(originalOpSpecs);
+    List<OperatorSpec> clonedList = new ArrayList<>(clonedOpSpecs);
+    Collections.sort(originalList, Comparator.comparing(OperatorSpec::getOpId));
+    Collections.sort(clonedList, Comparator.comparing(OperatorSpec::getOpId));
+    Iterator<OperatorSpec> oIter = originalList.iterator();
+    Iterator<OperatorSpec> nIter = clonedList.iterator();
+    oIter.forEachRemaining(opSpec -> assertClonedOpSpec(opSpec, nIter.next()));
+  }
+
+  private static void assertClonedOpSpec(OperatorSpec oOpSpec, OperatorSpec nOpSpec) {
+    assertNotEquals(oOpSpec, nOpSpec);
+    assertEquals(oOpSpec.getOpId(), nOpSpec.getOpId());
+    assertEquals(oOpSpec.getOpCode(), nOpSpec.getOpCode());
+    assertTimerFnsNotEqual(oOpSpec.getTimerFn(), nOpSpec.getTimerFn());
+    assertWatermarkFnNotEqual(nOpSpec.getWatermarkFn(), nOpSpec.getWatermarkFn());
+    assertAllOperators(oOpSpec.getRegisteredOperatorSpecs(), nOpSpec.getRegisteredOperatorSpecs());
+  }
+
+  private static void assertWatermarkFnNotEqual(WatermarkFunction watermarkFn, WatermarkFunction watermarkFn1) {
+    if (watermarkFn == watermarkFn1 && watermarkFn == null) {
+      return;
+    }
+    assertNotEquals(watermarkFn, watermarkFn1);
+  }
+
+  private static void assertTimerFnsNotEqual(TimerFunction timerFn, TimerFunction timerFn1) {
+    if (timerFn == timerFn1 && timerFn == null) {
+      return;
+    }
+    assertNotEquals(timerFn, timerFn1);
+  }
+
+  private static void assertClonedTables(Map<TableSpec, TableImpl> originalTables, Map<TableSpec, TableImpl> clonedTables) {
+    assertEquals(originalTables.size(), clonedTables.size());
+    assertEquals(originalTables.keySet(), clonedTables.keySet());
+    Iterator<TableImpl> oIter = originalTables.values().iterator();
+    Iterator<TableImpl> nIter = clonedTables.values().iterator();
+    oIter.forEachRemaining(oTable -> assertClonedTableImpl(oTable, nIter.next()));
+  }
+
+  private static void assertClonedTableImpl(TableImpl oTable, TableImpl nTable) {
+    assertNotEquals(oTable, nTable);
+    assertEquals(oTable.getTableSpec(), nTable.getTableSpec());
+  }
+
+  private static void assertClonedOutputs(Map<StreamSpec, OutputStreamImpl> originalOutputs,
+      Map<StreamSpec, OutputStreamImpl> clonedOutputs) {
+    assertEquals(originalOutputs.size(), clonedOutputs.size());
+    assertEquals(originalOutputs.keySet(), clonedOutputs.keySet());
+    Iterator<OutputStreamImpl> oIter = originalOutputs.values().iterator();
+    Iterator<OutputStreamImpl> nIter = clonedOutputs.values().iterator();
+    oIter.forEachRemaining(oOutput -> assertClonedOutputImpl(oOutput, nIter.next()));
+  }
+
+  private static void assertClonedOutputImpl(OutputStreamImpl oOutput, OutputStreamImpl nOutput) {
+    assertNotEquals(oOutput, nOutput);
+    assertEquals(oOutput.isKeyed(), nOutput.isKeyed());
+    assertEquals(oOutput.getSystemStream(), nOutput.getSystemStream());
+    assertEquals(oOutput.getStreamSpec(), nOutput.getStreamSpec());
+  }
+
+  private static void assertClonedInputs(Map<StreamSpec, InputOperatorSpec> originalInputs,
+      Map<StreamSpec, InputOperatorSpec> clonedInputs) {
+    assertEquals(originalInputs.size(), clonedInputs.size());
+    assertEquals(originalInputs.keySet(), clonedInputs.keySet());
+    Iterator<InputOperatorSpec> oIter = originalInputs.values().iterator();
+    Iterator<InputOperatorSpec> nIter = clonedInputs.values().iterator();
+    oIter.forEachRemaining(inputOp -> assertClonedInputOp(inputOp, nIter.next()));
+  }
+
+  private static void assertClonedInputOp(InputOperatorSpec originalInput, InputOperatorSpec clonedInput) {
+    assertNotEquals(originalInput, clonedInput);
+    assertEquals(originalInput.getOpId(), clonedInput.getOpId());
+    assertEquals(originalInput.getOpCode(), clonedInput.getOpCode());
+    assertEquals(originalInput.getStreamSpec(), clonedInput.getStreamSpec());
+    assertEquals(originalInput.isKeyed(), clonedInput.isKeyed());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/operators/spec/TestOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/spec/TestOperatorSpec.java b/samza-core/src/test/java/org/apache/samza/operators/spec/TestOperatorSpec.java
new file mode 100644
index 0000000..cb221b0
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/operators/spec/TestOperatorSpec.java
@@ -0,0 +1,465 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.operators.KV;
+import org.apache.samza.operators.TimerRegistry;
+import org.apache.samza.operators.data.TestMessageEnvelope;
+import org.apache.samza.operators.data.TestOutputMessageEnvelope;
+import org.apache.samza.operators.functions.FilterFunction;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.functions.JoinFunction;
+import org.apache.samza.operators.functions.MapFunction;
+import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.operators.functions.StreamTableJoinFunction;
+import org.apache.samza.operators.functions.TimerFunction;
+import org.apache.samza.operators.functions.WatermarkFunction;
+import org.apache.samza.serializers.JsonSerdeV2;
+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.table.TableSpec;
+import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+
+/**
+ * Test for all {@link OperatorSpec}
+ */
+public class TestOperatorSpec {
+
+  private static class MapWithWatermarkFn implements MapFunction<TestMessageEnvelope, TestOutputMessageEnvelope>, WatermarkFunction<TestOutputMessageEnvelope> {
+
+    @Override
+    public Collection<TestOutputMessageEnvelope> processWatermark(long watermark) {
+      return null;
+    }
+
+    @Override
+    public Long getOutputWatermark() {
+      return null;
+    }
+
+    @Override
+    public TestOutputMessageEnvelope apply(TestMessageEnvelope m) {
+      return new TestOutputMessageEnvelope(m.getKey(), m.getMessage().hashCode());
+    }
+  }
+
+  private static class MapWithTimerFn implements MapFunction<TestMessageEnvelope, TestOutputMessageEnvelope>, TimerFunction<String, TestOutputMessageEnvelope> {
+
+    @Override
+    public TestOutputMessageEnvelope apply(TestMessageEnvelope m) {
+      return new TestOutputMessageEnvelope(m.getKey(), m.getMessage().hashCode());
+    }
+
+    @Override
+    public void registerTimer(TimerRegistry<String> timerRegistry) {
+
+    }
+
+    @Override
+    public Collection<TestOutputMessageEnvelope> onTimer(String key, long timestamp) {
+      return null;
+    }
+  }
+
+  private static class MapWithEnum implements MapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> {
+    private OperatorSpecTestUtils.TestEnum type;
+
+    MapWithEnum(OperatorSpecTestUtils.TestEnum type) {
+      this.type = type;
+    }
+
+    OperatorSpecTestUtils.TestEnum getType() {
+      return this.type;
+    }
+
+    void setType(OperatorSpecTestUtils.TestEnum type) {
+      this.type = type;
+    }
+
+    @Override
+    public TestOutputMessageEnvelope apply(TestMessageEnvelope m) {
+      return new TestOutputMessageEnvelope(m.getKey(), m.getMessage().hashCode());
+    }
+  }
+
+  private static class TestJoinFunction implements JoinFunction<String, Object, Object, TestOutputMessageEnvelope> {
+    @Override
+    public TestOutputMessageEnvelope apply(Object message, Object otherMessage) {
+      return new TestOutputMessageEnvelope(message.toString(), message.hashCode() + otherMessage.hashCode());
+    }
+
+    @Override
+    public String getFirstKey(Object message) {
+      return message.toString();
+    }
+
+    @Override
+    public String getSecondKey(Object message) {
+      return message.toString();
+    }
+  }
+
+  private static class TestStreamTableJoinFunction implements StreamTableJoinFunction<String, Object, Object, TestOutputMessageEnvelope> {
+    @Override
+    public TestOutputMessageEnvelope apply(Object message, Object record) {
+      return new TestOutputMessageEnvelope(message.toString(), message.hashCode() + record.hashCode());
+    }
+
+    @Override
+    public String getMessageKey(Object message) {
+      return message.toString();
+    }
+
+    @Override
+    public String getRecordKey(Object record) {
+      return record.toString();
+    }
+  }
+
+  @Test
+  public void testStreamOperatorSpecWithFlatMap() {
+    FlatMapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> flatMap = m -> {
+      List<TestOutputMessageEnvelope> result = new ArrayList<>();
+      result.add(new TestOutputMessageEnvelope(m.getKey(), m.getMessage().hashCode()));
+      return result;
+    };
+    StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> streamOperatorSpec =
+        OperatorSpecs.createFlatMapOperatorSpec(flatMap, "op0");
+    StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> cloneOperatorSpec =
+        (StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope>) OperatorSpecTestUtils.copyOpSpec(streamOperatorSpec);
+    assertNotEquals(streamOperatorSpec, cloneOperatorSpec);
+    assertTrue(streamOperatorSpec.isClone(cloneOperatorSpec));
+    assertNotEquals(streamOperatorSpec.getTransformFn(), cloneOperatorSpec.getTransformFn());
+    assertTrue(cloneOperatorSpec.getTransformFn() instanceof FlatMapFunction);
+    assertNull(streamOperatorSpec.getWatermarkFn());
+    assertNull(cloneOperatorSpec.getWatermarkFn());
+    assertNull(streamOperatorSpec.getTimerFn());
+    assertNull(cloneOperatorSpec.getTimerFn());
+  }
+
+  @Test
+  public void testStreamOperatorSpecWithMap() {
+    MapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> mapFn =
+        m -> new TestOutputMessageEnvelope(m.getKey(), m.getMessage().hashCode());
+    StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> streamOperatorSpec =
+        OperatorSpecs.createMapOperatorSpec(mapFn, "op0");
+    StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> cloneOperatorSpec =
+        (StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope>) OperatorSpecTestUtils.copyOpSpec(streamOperatorSpec);
+    assertNotEquals(streamOperatorSpec, cloneOperatorSpec);
+    assertTrue(streamOperatorSpec.isClone(cloneOperatorSpec));
+    MapFunction userFn = (MapFunction) Whitebox.getInternalState(streamOperatorSpec, "mapFn");
+    assertEquals(userFn, mapFn);
+    assertNotEquals(streamOperatorSpec.getTransformFn(), cloneOperatorSpec.getTransformFn());
+    MapFunction clonedUserFn = (MapFunction) Whitebox.getInternalState(cloneOperatorSpec, "mapFn");
+    assertTrue(cloneOperatorSpec.getTransformFn() instanceof FlatMapFunction);
+    assertTrue(clonedUserFn instanceof MapFunction);
+    assertNotEquals(userFn, clonedUserFn);
+    assertNull(streamOperatorSpec.getWatermarkFn());
+    assertNull(cloneOperatorSpec.getWatermarkFn());
+    assertNull(streamOperatorSpec.getTimerFn());
+    assertNull(cloneOperatorSpec.getTimerFn());
+  }
+
+  @Test
+  public void testStreamOperatorSpecWithFilter() {
+    FilterFunction<TestMessageEnvelope> filterFn = m -> m.getKey().equals("key1");
+    StreamOperatorSpec<TestMessageEnvelope, TestMessageEnvelope> streamOperatorSpec =
+        OperatorSpecs.createFilterOperatorSpec(filterFn, "op0");
+    StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> cloneOperatorSpec =
+        (StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope>) OperatorSpecTestUtils.copyOpSpec(streamOperatorSpec);
+    assertNotEquals(streamOperatorSpec, cloneOperatorSpec);
+    assertTrue(streamOperatorSpec.isClone(cloneOperatorSpec));
+    FilterFunction userFn = (FilterFunction) Whitebox.getInternalState(streamOperatorSpec, "filterFn");
+    assertEquals(userFn, filterFn);
+    assertNotEquals(streamOperatorSpec.getTransformFn(), cloneOperatorSpec.getTransformFn());
+    FilterFunction clonedUserFn = (FilterFunction) Whitebox.getInternalState(cloneOperatorSpec, "filterFn");
+    assertTrue(cloneOperatorSpec.getTransformFn() instanceof FlatMapFunction);
+    assertTrue(clonedUserFn instanceof FilterFunction);
+    assertNotEquals(userFn, clonedUserFn);
+    assertNull(streamOperatorSpec.getWatermarkFn());
+    assertNull(cloneOperatorSpec.getWatermarkFn());
+    assertNull(streamOperatorSpec.getTimerFn());
+    assertNull(cloneOperatorSpec.getTimerFn());
+  }
+
+  @Test
+  public void testInputOperatorSpec() {
+    Serde<Object> objSerde = new Serde<Object>() {
+
+      @Override
+      public Object fromBytes(byte[] bytes) {
+        return null;
+      }
+
+      @Override
+      public byte[] toBytes(Object object) {
+        return new byte[0];
+      }
+    };
+
+    StreamSpec mockStreamSpec = mock(StreamSpec.class);
+    InputOperatorSpec<String, Object> inputOperatorSpec = new InputOperatorSpec<>(
+        mockStreamSpec, new StringSerde("UTF-8"), objSerde, true, "op0");
+    InputOperatorSpec<String, Object> inputOpCopy = (InputOperatorSpec<String, Object>) OperatorSpecTestUtils.copyOpSpec(inputOperatorSpec);
+
+    assertNotEquals("Expected deserialized copy of operator spec should not be the same as the original operator spec", inputOperatorSpec, inputOpCopy);
+    assertTrue(inputOperatorSpec.isClone(inputOpCopy));
+
+  }
+
+  @Test
+  public void testOutputOperatorSpec() {
+    Serde<Object> objSerde = new Serde<Object>() {
+
+      @Override
+      public Object fromBytes(byte[] bytes) {
+        return null;
+      }
+
+      @Override
+      public byte[] toBytes(Object object) {
+        return new byte[0];
+      }
+    };
+    StreamSpec mockStreamSpec = mock(StreamSpec.class);
+    OutputStreamImpl<KV<String, Object>> outputStrmImpl = new OutputStreamImpl<>(mockStreamSpec, new StringSerde("UTF-8"), objSerde, true);
+    OutputOperatorSpec<KV<String, Object>> outputOperatorSpec = new OutputOperatorSpec<KV<String, Object>>(outputStrmImpl, "op0");
+    OutputOperatorSpec<KV<String, Object>> outputOpCopy = (OutputOperatorSpec<KV<String, Object>>) OperatorSpecTestUtils
+        .copyOpSpec(outputOperatorSpec);
+    assertNotEquals("Expected deserialized copy of operator spec should not be the same as the original operator spec", outputOperatorSpec, outputOpCopy);
+    assertTrue(outputOperatorSpec.isClone(outputOpCopy));
+  }
+
+  @Test
+  public void testSinkOperatorSpec() {
+    SinkFunction<TestMessageEnvelope> sinkFn = (m, c, tc) -> System.out.print(m.toString());
+    SinkOperatorSpec<TestMessageEnvelope> sinkOpSpec = new SinkOperatorSpec<>(sinkFn, "op0");
+    SinkOperatorSpec<TestMessageEnvelope> sinkOpCopy = (SinkOperatorSpec<TestMessageEnvelope>) OperatorSpecTestUtils.copyOpSpec(sinkOpSpec);
+    assertNotEquals("Expected deserialized copy of operator spec should not be the same as the original operator spec", sinkOpSpec, sinkOpCopy);
+    assertTrue(sinkOpSpec.isClone(sinkOpCopy));
+  }
+
+  @Test
+  public void testJoinOperatorSpec() {
+
+    InputOperatorSpec<TestMessageEnvelope, Object> leftOpSpec = new InputOperatorSpec<>(
+        new StreamSpec("test-input-1", "test-input-1", "kafka"), new NoOpSerde<>(),
+        new NoOpSerde<>(), false, "op0");
+    InputOperatorSpec<TestMessageEnvelope, Object> rightOpSpec = new InputOperatorSpec<>(
+        new StreamSpec("test-input-2", "test-input-2", "kafka"), new NoOpSerde<>(),
+        new NoOpSerde<>(), false, "op1");
+
+    Serde<Object> objSerde = new Serde<Object>() {
+
+      @Override
+      public Object fromBytes(byte[] bytes) {
+        return null;
+      }
+
+      @Override
+      public byte[] toBytes(Object object) {
+        return new byte[0];
+      }
+    };
+
+    JoinFunction<String, Object, Object, TestOutputMessageEnvelope> joinFn = new TestJoinFunction();
+    JoinOperatorSpec<String, Object, Object, TestOutputMessageEnvelope> joinOperatorSpec =
+        new JoinOperatorSpec<>(leftOpSpec, rightOpSpec, joinFn, new StringSerde("UTF-8"), objSerde, objSerde, 50000, "op2");
+    JoinOperatorSpec<String, Object, Object, TestOutputMessageEnvelope> joinOpCopy =
+        (JoinOperatorSpec<String, Object, Object, TestOutputMessageEnvelope>) OperatorSpecTestUtils.copyOpSpec(joinOperatorSpec);
+    assertNotEquals("Expected deserialized copy of operator spec should not be the same as the original operator spec", joinOperatorSpec, joinOpCopy);
+    assertTrue(joinOperatorSpec.isClone(joinOpCopy));
+    assertTrue(joinOpCopy.getLeftInputOpSpec().isClone(leftOpSpec));
+    assertTrue(joinOpCopy.getRightInputOpSpec().isClone(rightOpSpec));
+  }
+
+  @Test
+  public void testStreamTableJoinOperatorSpec() {
+    StreamTableJoinFunction<String, Object, Object, TestOutputMessageEnvelope> joinFn = new TestStreamTableJoinFunction();
+
+    TableSpec tableSpec = new TableSpec("table-0", KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>()), "my.table.provider.class",
+        new MapConfig(new HashMap<String, String>() { { this.put("config1", "value1"); this.put("config2", "value2"); } }));
+
+    StreamTableJoinOperatorSpec<String, Object, Object, TestOutputMessageEnvelope> joinOperatorSpec =
+        new StreamTableJoinOperatorSpec<>(tableSpec, joinFn, "join-3");
+
+    StreamTableJoinOperatorSpec<String, Object, Object, TestOutputMessageEnvelope> joinOpSpecCopy =
+        (StreamTableJoinOperatorSpec<String, Object, Object, TestOutputMessageEnvelope>) OperatorSpecTestUtils.copyOpSpec(joinOperatorSpec);
+    assertNotEquals(joinOpSpecCopy, joinOperatorSpec);
+    assertEquals(joinOpSpecCopy.getOpId(), joinOperatorSpec.getOpId());
+    assertTrue(joinOpSpecCopy.getTableSpec() != joinOperatorSpec.getTableSpec());
+    assertEquals(joinOpSpecCopy.getTableSpec().getId(), joinOperatorSpec.getTableSpec().getId());
+    assertEquals(joinOpSpecCopy.getTableSpec().getTableProviderFactoryClassName(), joinOperatorSpec.getTableSpec().getTableProviderFactoryClassName());
+  }
+
+  @Test
+  public void testSendToTableOperatorSpec() {
+    TableSpec tableSpec = new TableSpec("table-0", KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>()), "my.table.provider.class",
+        new MapConfig(new HashMap<String, String>() { { this.put("config1", "value1"); this.put("config2", "value2"); } }));
+    SendToTableOperatorSpec<String, Integer> sendOpSpec =
+        new SendToTableOperatorSpec<>(tableSpec, "output-1");
+    SendToTableOperatorSpec<String, Integer> sendToCopy = (SendToTableOperatorSpec<String, Integer>) OperatorSpecTestUtils
+        .copyOpSpec(sendOpSpec);
+    assertNotEquals(sendToCopy, sendOpSpec);
+    assertEquals(sendToCopy.getOpId(), sendOpSpec.getOpId());
+    assertTrue(sendToCopy.getTableSpec() != sendOpSpec.getTableSpec() && sendToCopy.getTableSpec().equals(sendOpSpec.getTableSpec()));
+  }
+
+  @Test
+  public void testBroadcastOperatorSpec() {
+    OutputStreamImpl<TestOutputMessageEnvelope> outputStream =
+        new OutputStreamImpl<>(new StreamSpec("output-0", "outputStream-0", "kafka"), new StringSerde("UTF-8"), new JsonSerdeV2<TestOutputMessageEnvelope>(), true);
+    BroadcastOperatorSpec<TestOutputMessageEnvelope> broadcastOpSpec = new BroadcastOperatorSpec<>(outputStream, "broadcast-1");
+    BroadcastOperatorSpec<TestOutputMessageEnvelope> broadcastOpCopy = (BroadcastOperatorSpec<TestOutputMessageEnvelope>) OperatorSpecTestUtils
+        .copyOpSpec(broadcastOpSpec);
+    assertNotEquals(broadcastOpCopy, broadcastOpSpec);
+    assertEquals(broadcastOpCopy.getOpId(), broadcastOpSpec.getOpId());
+    assertTrue(broadcastOpCopy.getOutputStream() != broadcastOpSpec.getOutputStream());
+    assertEquals(broadcastOpCopy.getOutputStream().getSystemStream(), broadcastOpSpec.getOutputStream().getSystemStream());
+    assertEquals(broadcastOpCopy.getOutputStream().isKeyed(), broadcastOpSpec.getOutputStream().isKeyed());
+  }
+
+  @Test
+  public void testMapStreamOperatorSpecWithWatermark() {
+    MapWithWatermarkFn testMapFn = new MapWithWatermarkFn();
+
+    StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> streamOperatorSpec =
+        OperatorSpecs.createMapOperatorSpec(testMapFn, "op0");
+    StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> cloneOperatorSpec =
+        (StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope>) OperatorSpecTestUtils.copyOpSpec(streamOperatorSpec);
+    assertNotEquals(streamOperatorSpec, cloneOperatorSpec);
+    assertTrue(streamOperatorSpec.isClone(cloneOperatorSpec));
+    assertNotEquals(streamOperatorSpec.getTransformFn(), cloneOperatorSpec.getTransformFn());
+    assertEquals(streamOperatorSpec.getWatermarkFn(), testMapFn);
+    assertNotNull(cloneOperatorSpec.getWatermarkFn());
+    assertNotEquals(cloneOperatorSpec.getTransformFn(), cloneOperatorSpec.getWatermarkFn());
+    assertNull(streamOperatorSpec.getTimerFn());
+    assertNull(cloneOperatorSpec.getTimerFn());
+  }
+
+  @Test
+  public void testMapStreamOperatorSpecWithTimer() {
+    MapWithTimerFn testMapFn = new MapWithTimerFn();
+
+    StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> streamOperatorSpec =
+        OperatorSpecs.createMapOperatorSpec(testMapFn, "op0");
+    StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> cloneOperatorSpec =
+        (StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope>) OperatorSpecTestUtils.copyOpSpec(streamOperatorSpec);
+    assertNotEquals(streamOperatorSpec, cloneOperatorSpec);
+    assertTrue(streamOperatorSpec.isClone(cloneOperatorSpec));
+    assertNotEquals(streamOperatorSpec.getTransformFn(), cloneOperatorSpec.getTransformFn());
+    assertNull(streamOperatorSpec.getWatermarkFn());
+    assertNull(cloneOperatorSpec.getWatermarkFn());
+    assertNotEquals(cloneOperatorSpec.getTransformFn(), cloneOperatorSpec.getWatermarkFn());
+    assertEquals(streamOperatorSpec.getTimerFn(), testMapFn);
+    assertNotNull(cloneOperatorSpec.getTimerFn());
+    assertNotEquals(streamOperatorSpec.getTimerFn(), cloneOperatorSpec.getTimerFn());
+  }
+
+  @Test
+  public void testStreamOperatorSpecWithMapAndListInClosure() {
+    List<Integer> integers = new ArrayList<>(1);
+    integers.add(0, 100);
+    List<String> keys = new ArrayList<>(1);
+    keys.add(0, "test-1");
+    MapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> mapFn =
+        m -> new TestOutputMessageEnvelope(keys.get(m.getKey().hashCode() % 1), integers.get(m.getMessage().hashCode() % 1));
+    StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> streamOperatorSpec =
+        OperatorSpecs.createMapOperatorSpec(mapFn, "op0");
+    StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> cloneOperatorSpec =
+        (StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope>) OperatorSpecTestUtils.copyOpSpec(streamOperatorSpec);
+    assertNotEquals(streamOperatorSpec, cloneOperatorSpec);
+    assertTrue(streamOperatorSpec.isClone(cloneOperatorSpec));
+    MapFunction userFn = (MapFunction) Whitebox.getInternalState(streamOperatorSpec, "mapFn");
+    assertEquals(userFn, mapFn);
+    assertNotEquals(streamOperatorSpec.getTransformFn(), cloneOperatorSpec.getTransformFn());
+    MapFunction clonedUserFn = (MapFunction) Whitebox.getInternalState(cloneOperatorSpec, "mapFn");
+    assertTrue(cloneOperatorSpec.getTransformFn() instanceof FlatMapFunction);
+    assertTrue(clonedUserFn instanceof MapFunction);
+    assertNotEquals(userFn, clonedUserFn);
+
+    // verify changing the values in the original keys and integers list will change the result of the original map function
+    TestMessageEnvelope mockImsg = new TestMessageEnvelope("input-key-x", new String("value-x"));
+    assertEquals(((MapFunction) userFn).apply(mockImsg), new TestOutputMessageEnvelope("test-1", 100));
+    integers.set(0, 200);
+    keys.set(0, "test-2");
+    assertEquals(((MapFunction) userFn).apply(mockImsg), new TestOutputMessageEnvelope("test-2", 200));
+    // verify that the cloned map function uses a different copy of lists and still yields the same result
+    assertEquals(((MapFunction) clonedUserFn).apply(mockImsg), new TestOutputMessageEnvelope("test-1", 100));
+  }
+
+  @Test
+  public void testStreamOperatorSpecWithMapWithFunctionReference() {
+    MapFunction<KV<String, Object>, Object> mapFn = KV::getValue;
+    StreamOperatorSpec<KV<String, Object>, Object> streamOperatorSpec =
+        OperatorSpecs.createMapOperatorSpec(mapFn, "op0");
+    StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> cloneOperatorSpec =
+        (StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope>) OperatorSpecTestUtils.copyOpSpec(streamOperatorSpec);
+    assertNotEquals(streamOperatorSpec, cloneOperatorSpec);
+    assertTrue(streamOperatorSpec.isClone(cloneOperatorSpec));
+    MapFunction userFn = (MapFunction) Whitebox.getInternalState(streamOperatorSpec, "mapFn");
+    assertEquals(userFn, mapFn);
+    assertNotEquals(streamOperatorSpec.getTransformFn(), cloneOperatorSpec.getTransformFn());
+    MapFunction clonedUserFn = (MapFunction) Whitebox.getInternalState(cloneOperatorSpec, "mapFn");
+    assertTrue(cloneOperatorSpec.getTransformFn() instanceof FlatMapFunction);
+    assertTrue(clonedUserFn instanceof MapFunction);
+    assertNotEquals(userFn, clonedUserFn);
+  }
+
+  @Test
+  public void testStreamOperatorSpecWithMapWithEnum() {
+    MapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> mapFn = new MapWithEnum(OperatorSpecTestUtils.TestEnum.One);
+    StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> streamOperatorSpec =
+        OperatorSpecs.createMapOperatorSpec(mapFn, "op0");
+    assertTrue(streamOperatorSpec instanceof MapOperatorSpec);
+    StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> cloneOperatorSpec =
+        (StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope>) OperatorSpecTestUtils.copyOpSpec(streamOperatorSpec);
+    assertNotEquals(streamOperatorSpec, cloneOperatorSpec);
+    assertTrue(streamOperatorSpec.isClone(cloneOperatorSpec));
+    MapFunction userFn = (MapFunction) Whitebox.getInternalState(streamOperatorSpec, "mapFn");
+    assertEquals(userFn, mapFn);
+    assertNotEquals(streamOperatorSpec.getTransformFn(), cloneOperatorSpec.getTransformFn());
+    MapFunction clonedUserFn = (MapFunction) Whitebox.getInternalState(cloneOperatorSpec, "mapFn");
+    assertTrue(cloneOperatorSpec.getTransformFn() instanceof FlatMapFunction);
+    assertTrue(clonedUserFn instanceof MapWithEnum);
+    assertNotEquals(userFn, clonedUserFn);
+    // originally the types should be the same
+    assertTrue(((MapWithEnum) userFn).getType() == ((MapWithEnum) clonedUserFn).getType());
+    // after changing the type of the cloned user function, the types are different now
+    ((MapWithEnum) clonedUserFn).setType(OperatorSpecTestUtils.TestEnum.Two);
+    assertTrue(((MapWithEnum) userFn).getType() != ((MapWithEnum) clonedUserFn).getType());
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java b/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java
new file mode 100644
index 0000000..00ec176
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/operators/spec/TestPartitionByOperatorSpec.java
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.spec;
+
+import java.util.Collection;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.operators.KV;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.OperatorSpecGraph;
+import org.apache.samza.operators.StreamGraphSpec;
+import org.apache.samza.operators.TimerRegistry;
+import org.apache.samza.operators.functions.MapFunction;
+import org.apache.samza.operators.functions.TimerFunction;
+import org.apache.samza.operators.functions.WatermarkFunction;
+import org.apache.samza.runtime.ApplicationRunner;
+import org.apache.samza.serializers.NoOpSerde;
+import org.apache.samza.system.StreamSpec;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+
+/**
+ * Unit tests for partitionBy operator
+ */
+public class TestPartitionByOperatorSpec {
+
+  private final ApplicationRunner mockRunner = mock(ApplicationRunner.class);
+  private final Config mockConfig = mock(Config.class);
+  private final String testInputId = "test-input-1";
+  private final String testJobName = "testJob";
+  private final String testJobId = "1";
+  private final String testReparStreamName = "parByKey";
+  private StreamGraphSpec graphSpec = null;
+
+  class TimerMapFn implements MapFunction<Object, String>, TimerFunction<String, Object> {
+
+    @Override
+    public String apply(Object message) {
+      return message.toString();
+    }
+
+    @Override
+    public void registerTimer(TimerRegistry<String> timerRegistry) {
+
+    }
+
+    @Override
+    public Collection<Object> onTimer(String key, long timestamp) {
+      return null;
+    }
+  }
+
+  class WatermarkMapFn implements MapFunction<Object, String>, WatermarkFunction<Object> {
+
+    @Override
+    public String apply(Object message) {
+      return message.toString();
+    }
+
+    @Override
+    public Collection<Object> processWatermark(long watermark) {
+      return null;
+    }
+
+    @Override
+    public Long getOutputWatermark() {
+      return null;
+    }
+  }
+
+  @Before
+  public void setup() {
+    when(mockConfig.get(JobConfig.JOB_NAME())).thenReturn(testJobName);
+    when(mockConfig.get(JobConfig.JOB_ID(), "1")).thenReturn(testJobId);
+    StreamSpec inputSpec1 = new StreamSpec(testInputId, testInputId, "kafka");
+    when(mockRunner.getStreamSpec(testInputId)).thenReturn(inputSpec1);
+    String intermediateStreamName = String.format("%s-%s-partition_by-%s", testJobName, testJobId, testReparStreamName);
+    StreamSpec intermediateSpec1 = new StreamSpec(intermediateStreamName, intermediateStreamName, "kafka");
+    when(mockRunner.getStreamSpec(intermediateStreamName)).thenReturn(intermediateSpec1);
+    graphSpec = new StreamGraphSpec(mockRunner, mockConfig);
+  }
+
+  @Test
+  public void testPartitionBy() {
+    MessageStream inputStream = graphSpec.getInputStream(testInputId);
+    MapFunction<Object, String> keyFn = m -> m.toString();
+    MapFunction<Object, Object> valueFn = m -> m;
+    MessageStream<KV<String, Object>>
+        reparStream = inputStream.partitionBy(keyFn, valueFn, testReparStreamName);
+    InputOperatorSpec inputOpSpec = (InputOperatorSpec) Whitebox.getInternalState(reparStream, "operatorSpec");
+    assertEquals(inputOpSpec.getStreamSpec().getId(), String.format("%s-%s-partition_by-%s", testJobName, testJobId, testReparStreamName));
+    assertTrue(inputOpSpec.getKeySerde() instanceof NoOpSerde);
+    assertTrue(inputOpSpec.getValueSerde() instanceof NoOpSerde);
+    assertTrue(inputOpSpec.isKeyed());
+    assertNull(inputOpSpec.getTimerFn());
+    assertNull(inputOpSpec.getWatermarkFn());
+    InputOperatorSpec originInputSpec = (InputOperatorSpec) Whitebox.getInternalState(inputStream, "operatorSpec");
+    assertTrue(originInputSpec.getRegisteredOperatorSpecs().toArray()[0] instanceof PartitionByOperatorSpec);
+    PartitionByOperatorSpec reparOpSpec  = (PartitionByOperatorSpec) originInputSpec.getRegisteredOperatorSpecs().toArray()[0];
+    assertEquals(reparOpSpec.getOpId(), String.format("%s-%s-partition_by-%s", testJobName, testJobId, testReparStreamName));
+    assertEquals(reparOpSpec.getKeyFunction(), keyFn);
+    assertEquals(reparOpSpec.getValueFunction(), valueFn);
+    assertEquals(reparOpSpec.getOutputStream().getStreamSpec(), new StreamSpec(reparOpSpec.getOpId(), reparOpSpec.getOpId(), "kafka"));
+    assertNull(reparOpSpec.getTimerFn());
+    assertNull(reparOpSpec.getWatermarkFn());
+  }
+
+  @Test
+  public void testCopy() {
+    MessageStream inputStream = graphSpec.getInputStream(testInputId);
+    inputStream.partitionBy(m -> m.toString(), m -> m, testReparStreamName);
+    OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph();
+    OperatorSpecGraph clonedGraph = specGraph.clone();
+    OperatorSpecTestUtils.assertClonedGraph(specGraph, clonedGraph);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testTimerFunctionAsKeyFn() {
+    TimerMapFn keyFn = new TimerMapFn();
+    MessageStream<Object> inputStream = graphSpec.getInputStream(testInputId);
+    inputStream.partitionBy(keyFn, m -> m, "parByKey");
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testWatermarkFunctionAsKeyFn() {
+    WatermarkMapFn keyFn = new WatermarkMapFn();
+    MessageStream<Object> inputStream = graphSpec.getInputStream(testInputId);
+    inputStream.partitionBy(keyFn, m -> m, "parByKey");
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testTimerFunctionAsValueFn() {
+    TimerMapFn valueFn = new TimerMapFn();
+    MessageStream<Object> inputStream = graphSpec.getInputStream(testInputId);
+    inputStream.partitionBy(m -> m.toString(), valueFn, "parByKey");
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testWatermarkFunctionAsValueFn() {
+    WatermarkMapFn valueFn = new WatermarkMapFn();
+    MessageStream<Object> inputStream = graphSpec.getInputStream(testInputId);
+    inputStream.partitionBy(m -> m.toString(), valueFn, "parByKey");
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/operators/spec/TestWindowOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/spec/TestWindowOperatorSpec.java b/samza-core/src/test/java/org/apache/samza/operators/spec/TestWindowOperatorSpec.java
index 65f1dc6..0a2214b 100644
--- a/samza-core/src/test/java/org/apache/samza/operators/spec/TestWindowOperatorSpec.java
+++ b/samza-core/src/test/java/org/apache/samza/operators/spec/TestWindowOperatorSpec.java
@@ -19,50 +19,312 @@
 
 package org.apache.samza.operators.spec;
 
+import org.apache.samza.operators.TimerRegistry;
+import org.apache.samza.operators.functions.TimerFunction;
+import org.apache.samza.operators.functions.WatermarkFunction;
 import org.apache.samza.serializers.Serde;
+import org.apache.samza.operators.functions.FoldLeftFunction;
+import org.apache.samza.operators.functions.MapFunction;
+import org.apache.samza.operators.functions.SupplierFunction;
 import org.apache.samza.operators.triggers.Trigger;
 import org.apache.samza.operators.triggers.Triggers;
 import org.apache.samza.operators.windows.internal.WindowInternal;
 import org.apache.samza.operators.windows.internal.WindowType;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
 
 import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
 
+import static org.junit.Assert.*;
 import static org.mockito.Mockito.mock;
 
 public class TestWindowOperatorSpec {
+
+  private Trigger defaultTrigger;
+  private Trigger earlyTrigger;
+  private Trigger lateTrigger;
+  private FoldLeftFunction<Object, Collection> foldFn;
+  private SupplierFunction<Collection> supplierFunction;
+  private MapFunction<Object, Object> keyFn;
+  private MapFunction<Object, Long> timeFn;
+
+  @Before
+  public void setup() {
+
+    foldFn = (m, c) -> {
+      c.add(m);
+      return c;
+    };
+    supplierFunction = () -> new ArrayList<>();
+    keyFn = m -> m.toString();
+    timeFn = m -> 123456L;
+
+    defaultTrigger = Triggers.timeSinceFirstMessage(Duration.ofMillis(150));
+    earlyTrigger = Triggers.repeat(Triggers.count(5));
+    lateTrigger = null;
+  }
+
   @Test
   public void testTriggerIntervalWithNestedTimeTriggers() {
-    Trigger defaultTrigger = Triggers.timeSinceFirstMessage(Duration.ofMillis(150));
-    Trigger lateTrigger = Triggers.any(Triggers.count(6), Triggers.timeSinceFirstMessage(Duration.ofMillis(15)));
-    Trigger earlyTrigger = Triggers.repeat(
-        Triggers.any(Triggers.count(23),
-            Triggers.timeSinceFirstMessage(Duration.ofMillis(15)),
-            Triggers.any(Triggers.any(Triggers.count(6),
-                Triggers.timeSinceFirstMessage(Duration.ofMillis(15)),
-                Triggers.timeSinceFirstMessage(Duration.ofMillis(25)),
-                Triggers.timeSinceLastMessage(Duration.ofMillis(15))))));
-
-    WindowInternal window = new WindowInternal(defaultTrigger, null, null, null,
-            null, WindowType.SESSION, null, null, mock(Serde.class));
-    window.setEarlyTrigger(earlyTrigger);
-    window.setLateTrigger(lateTrigger);
+    defaultTrigger = Triggers.timeSinceFirstMessage(Duration.ofMillis(150));
+    lateTrigger = Triggers.any(Triggers.count(6), Triggers.timeSinceFirstMessage(Duration.ofMillis(15)));
+    earlyTrigger = Triggers.repeat(
+      Triggers.any(Triggers.count(23),
+          Triggers.timeSinceFirstMessage(Duration.ofMillis(15)),
+          Triggers.any(Triggers.any(Triggers.count(6),
+              Triggers.timeSinceFirstMessage(Duration.ofMillis(15)),
+              Triggers.timeSinceFirstMessage(Duration.ofMillis(25)),
+              Triggers.timeSinceLastMessage(Duration.ofMillis(15))))));
 
-    WindowOperatorSpec spec = new WindowOperatorSpec(window, "0");
-    Assert.assertEquals(spec.getDefaultTriggerMs(), 5);
+    WindowOperatorSpec spec = getWindowOperatorSpec("w0");
+    assertEquals(spec.getDefaultTriggerMs(), 5);
   }
 
   @Test
   public void testTriggerIntervalWithSingleTimeTrigger() {
-    Trigger defaultTrigger = Triggers.timeSinceFirstMessage(Duration.ofMillis(150));
-    Trigger earlyTrigger = Triggers.repeat(Triggers.count(5));
+    WindowOperatorSpec spec = getWindowOperatorSpec("w0");
+    assertEquals(spec.getDefaultTriggerMs(), 150);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testIllegalTimerFunctionAsInitializer() {
+    class TimedSupplierFunction implements SupplierFunction<Collection>, TimerFunction<Object, Collection> {
+
+      @Override
+      public Collection get() {
+        return new ArrayList<>();
+      }
+
+      @Override
+      public void registerTimer(TimerRegistry<Object> timerRegistry) {
+
+      }
+
+      @Override
+      public Collection<Collection> onTimer(Object key, long timestamp) {
+        return null;
+      }
+    }
+    supplierFunction = new TimedSupplierFunction();
+
+    getWindowOperatorSpec("w0");
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testIllegalWatermarkFunctionAsInitializer() {
+    class WatermarkSupplierFunction implements SupplierFunction<Collection>, WatermarkFunction<Collection> {
+
+      @Override
+      public Collection get() {
+        return new ArrayList<>();
+      }
+
+      @Override
+      public Collection<Collection> processWatermark(long watermark) {
+        return null;
+      }
+
+      @Override
+      public Long getOutputWatermark() {
+        return null;
+      }
+    }
+    supplierFunction = new WatermarkSupplierFunction();
+
+    getWindowOperatorSpec("w0");
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testIllegalTimerFunctionAsKeyFn() {
+    class TimerMapFunction implements MapFunction<Object, Object>, TimerFunction<Object, Object> {
+
+      @Override
+      public Object apply(Object message) {
+        return message.toString();
+      }
+
+      @Override
+      public void registerTimer(TimerRegistry<Object> timerRegistry) {
+
+      }
+
+      @Override
+      public Collection<Object> onTimer(Object key, long timestamp) {
+        return null;
+      }
+    }
+    keyFn = new TimerMapFunction();
+
+    getWindowOperatorSpec("w0");
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testIllegalWatermarkFunctionAsKeyFn() {
+    class WatermarkMapFunction implements MapFunction<Object, Object>, WatermarkFunction<Object> {
+
+      @Override
+      public Object apply(Object message) {
+        return message.toString();
+      }
+
+      @Override
+      public Collection<Object> processWatermark(long watermark) {
+        return null;
+      }
+
+      @Override
+      public Long getOutputWatermark() {
+        return null;
+      }
+    }
+    keyFn = new WatermarkMapFunction();
+
+    getWindowOperatorSpec("w0");
+  }
 
-    WindowInternal window = new WindowInternal(defaultTrigger, null, null, null,
-            null, WindowType.SESSION, null, null, mock(Serde.class));
+  @Test(expected = IllegalArgumentException.class)
+  public void testIllegalTimerFunctionAsEventTimeFn() {
+    class TimerMapFunction implements MapFunction<Object, Long>, TimerFunction<Object, Object> {
+
+      @Override
+      public Long apply(Object message) {
+        return 123456L;
+      }
+
+      @Override
+      public void registerTimer(TimerRegistry<Object> timerRegistry) {
+
+      }
+
+      @Override
+      public Collection<Object> onTimer(Object key, long timestamp) {
+        return null;
+      }
+    }
+    timeFn = new TimerMapFunction();
+
+    getWindowOperatorSpec("w0");
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testIllegalWatermarkFunctionAsEventTimeFn() {
+    class WatermarkMapFunction implements MapFunction<Object, Long>, WatermarkFunction<Object> {
+
+      @Override
+      public Long apply(Object message) {
+        return 123456L;
+      }
+
+      @Override
+      public Collection<Object> processWatermark(long watermark) {
+        return null;
+      }
+
+      @Override
+      public Long getOutputWatermark() {
+        return null;
+      }
+    }
+    timeFn = new WatermarkMapFunction();
+
+    getWindowOperatorSpec("w0");
+  }
+
+  @Test
+  public void testTimerFunctionAsFoldLeftFn() {
+    class TimerFoldLeftFunction implements FoldLeftFunction<Object, Collection>, TimerFunction<Object, Collection> {
+
+      @Override
+      public Collection apply(Object message, Collection oldValue) {
+        oldValue.add(message);
+        return oldValue;
+      }
+
+      @Override
+      public void registerTimer(TimerRegistry<Object> timerRegistry) {
+
+      }
+
+      @Override
+      public Collection<Collection> onTimer(Object key, long timestamp) {
+        return null;
+      }
+    }
+
+    foldFn = new TimerFoldLeftFunction();
+    WindowOperatorSpec<Object, Object, Collection> windowSpec = getWindowOperatorSpec("w0");
+    assertEquals(windowSpec.getTimerFn(), foldFn);
+    assertNull(windowSpec.getWatermarkFn());
+  }
+
+  @Test
+  public void testWatermarkFunctionAsFoldLeftFn() {
+    class WatermarkFoldLeftFunction implements FoldLeftFunction<Object, Collection>, WatermarkFunction<Object> {
+
+      @Override
+      public Collection<Object> processWatermark(long watermark) {
+        return null;
+      }
+
+      @Override
+      public Long getOutputWatermark() {
+        return null;
+      }
+
+      @Override
+      public Collection apply(Object message, Collection oldValue) {
+        oldValue.add(message);
+        return oldValue;
+      }
+    }
+
+    foldFn = new WatermarkFoldLeftFunction();
+    WindowOperatorSpec<Object, Object, Collection> windowSpec = getWindowOperatorSpec("w0");
+    assertEquals(windowSpec.getWatermarkFn(), foldFn);
+    assertNull(windowSpec.getTimerFn());
+  }
+
+  @Test
+  public void testCopy() {
+    WindowInternal<Object, Object, Collection> window = new WindowInternal<Object, Object, Collection>(
+        defaultTrigger, supplierFunction, foldFn, keyFn, timeFn, WindowType.SESSION, null,
+        mock(Serde.class), mock(Serde.class));
     window.setEarlyTrigger(earlyTrigger);
 
-    WindowOperatorSpec spec = new WindowOperatorSpec(window, "0");
-    Assert.assertEquals(spec.getDefaultTriggerMs(), 150);
+    WindowOperatorSpec<Object, Object, Collection> spec = new WindowOperatorSpec<>(window, "w0");
+
+    WindowOperatorSpec<Object, Object, Collection> copy =
+        (WindowOperatorSpec<Object, Object, Collection>) OperatorSpecTestUtils.copyOpSpec(spec);
+
+    Assert.assertNotEquals(spec, copy);
+    Assert.assertTrue(spec.isClone(copy));
+    Assert.assertNotEquals(spec.getWindow(), copy.getWindow());
+    Assert.assertNotEquals(copy.getWindow().getInitializer(), supplierFunction);
+    assertEquals(copy.getWindow().getInitializer().get(), supplierFunction.get());
+    Assert.assertNotEquals(copy.getWindow().getFoldLeftFunction(), foldFn);
+    Object mockMsg = new Object();
+    assertEquals(copy.getWindow().getFoldLeftFunction().apply(mockMsg, new ArrayList<>()), foldFn.apply(mockMsg, new ArrayList<>()));
+    Assert.assertNotEquals(copy.getWindow().getKeyExtractor(), keyFn);
+    assertEquals(copy.getWindow().getKeyExtractor().apply(mockMsg), keyFn.apply(mockMsg));
+    Assert.assertNotEquals(copy.getWindow().getEventTimeExtractor(), timeFn);
+    assertEquals(copy.getWindow().getEventTimeExtractor().apply(mockMsg), timeFn.apply(mockMsg));
+    assertEquals(copy.getDefaultTriggerMs(), 150);
+  }
+
+  private WindowOperatorSpec getWindowOperatorSpec(String opId) {
+    WindowInternal<Object, Object, Collection> window = new WindowInternal<Object, Object, Collection>(
+        defaultTrigger, supplierFunction, foldFn, keyFn, timeFn, WindowType.SESSION, null,
+        mock(Serde.class), mock(Serde.class));
+    if (earlyTrigger != null) {
+      window.setEarlyTrigger(earlyTrigger);
+    }
+    if (lateTrigger != null) {
+      window.setLateTrigger(lateTrigger);
+    }
+    return new WindowOperatorSpec<>(window, opId);
   }
+
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/runtime/TestAbstractApplicationRunner.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestAbstractApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestAbstractApplicationRunner.java
index ed13b5b..b8d3f15 100644
--- a/samza-core/src/test/java/org/apache/samza/runtime/TestAbstractApplicationRunner.java
+++ b/samza-core/src/test/java/org/apache/samza/runtime/TestAbstractApplicationRunner.java
@@ -222,10 +222,10 @@ public class TestAbstractApplicationRunner {
                                       StreamConfig.SYSTEM(), TEST_SYSTEM);
 
     AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config);
-    StreamSpec spec = runner.getStreamSpec(STREAM_ID, TEST_PHYSICAL_NAME);
+    StreamSpec spec = runner.getStreamSpec(STREAM_ID);
 
     assertEquals(STREAM_ID, spec.getId());
-    assertEquals(TEST_PHYSICAL_NAME, spec.getPhysicalName());
+    assertEquals(TEST_PHYSICAL_NAME2, spec.getPhysicalName());
     assertEquals(TEST_SYSTEM, spec.getSystemName());
   }
 
@@ -233,11 +233,11 @@ public class TestAbstractApplicationRunner {
   @Test
   public void testGetStreamPhysicalNameArgSpecialCharacters() {
     Config config = buildStreamConfig(STREAM_ID,
-                                      StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME2,
+                                      StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME_SPECIAL_CHARS,
                                       StreamConfig.SYSTEM(), TEST_SYSTEM);
 
     AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config);
-    StreamSpec spec = runner.getStreamSpec(STREAM_ID, TEST_PHYSICAL_NAME_SPECIAL_CHARS);
+    StreamSpec spec = runner.getStreamSpec(STREAM_ID);
     assertEquals(TEST_PHYSICAL_NAME_SPECIAL_CHARS, spec.getPhysicalName());
   }
 
@@ -245,11 +245,11 @@ public class TestAbstractApplicationRunner {
   @Test
   public void testGetStreamPhysicalNameArgNull() {
     Config config = buildStreamConfig(STREAM_ID,
-                                      StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME2,
+                                      StreamConfig.PHYSICAL_NAME(), null,
                                       StreamConfig.SYSTEM(), TEST_SYSTEM);
 
     AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config);
-    StreamSpec spec = runner.getStreamSpec(STREAM_ID, null);
+    StreamSpec spec = runner.getStreamSpec(STREAM_ID);
     assertNull(spec.getPhysicalName());
   }
 
@@ -257,11 +257,11 @@ public class TestAbstractApplicationRunner {
   @Test
   public void testGetStreamSystemNameArgValid() {
     Config config = buildStreamConfig(STREAM_ID,
-                                      StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME2, // This should be ignored because of the explicit arg
-                                      StreamConfig.SYSTEM(), TEST_SYSTEM2);              // This too
+                                      StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME, // This should be ignored because of the explicit arg
+                                      StreamConfig.SYSTEM(), TEST_SYSTEM);              // This too
 
     AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config);
-    StreamSpec spec = runner.getStreamSpec(STREAM_ID, TEST_PHYSICAL_NAME, TEST_SYSTEM);
+    StreamSpec spec = runner.getStreamSpec(STREAM_ID);
 
     assertEquals(STREAM_ID, spec.getId());
     assertEquals(TEST_PHYSICAL_NAME, spec.getPhysicalName());
@@ -272,33 +272,33 @@ public class TestAbstractApplicationRunner {
   @Test(expected = IllegalArgumentException.class)
   public void testGetStreamSystemNameArgInvalid() {
     Config config = buildStreamConfig(STREAM_ID,
-                                      StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME2,
-                                      StreamConfig.SYSTEM(), TEST_SYSTEM2);
+                                      StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME,
+                                      StreamConfig.SYSTEM(), TEST_SYSTEM_INVALID);
 
     AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config);
-    runner.getStreamSpec(STREAM_ID, TEST_PHYSICAL_NAME, TEST_SYSTEM_INVALID);
+    runner.getStreamSpec(STREAM_ID);
   }
 
   // Empty strings are NOT allowed for system name, because it's used as an identifier in the config.
   @Test(expected = IllegalArgumentException.class)
   public void testGetStreamSystemNameArgEmpty() {
     Config config = buildStreamConfig(STREAM_ID,
-        StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME2,
-        StreamConfig.SYSTEM(), TEST_SYSTEM2);
+        StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME,
+        StreamConfig.SYSTEM(), "");
 
     AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config);
-    runner.getStreamSpec(STREAM_ID, TEST_PHYSICAL_NAME, "");
+    runner.getStreamSpec(STREAM_ID);
   }
 
   // Null is not allowed IllegalArgumentException system name.
   @Test(expected = IllegalArgumentException.class)
   public void testGetStreamSystemNameArgNull() {
     Config config = buildStreamConfig(STREAM_ID,
-                                      StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME2,
-                                      StreamConfig.SYSTEM(), TEST_SYSTEM2);
+                                      StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME,
+                                      StreamConfig.SYSTEM(), null);
 
     AbstractApplicationRunner runner = new TestAbstractApplicationRunnerImpl(config);
-    runner.getStreamSpec(STREAM_ID, TEST_PHYSICAL_NAME, null);
+    runner.getStreamSpec(STREAM_ID);
   }
 
   // Special characters are NOT allowed for streamId, because it's used as an identifier in the config.

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 84ecc6c..595dda2 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
@@ -29,6 +29,7 @@ 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.Config;
 import org.apache.samza.config.JobConfig;
 import org.apache.samza.config.JobCoordinatorConfig;
 import org.apache.samza.config.MapConfig;
@@ -76,7 +77,6 @@ public class TestLocalApplicationRunner {
     Map<String, String> config = new HashMap<>();
     LocalApplicationRunner runner = spy(new LocalApplicationRunner(new MapConfig(config)));
     StreamApplication app = mock(StreamApplication.class);
-    doNothing().when(app).init(anyObject(), anyObject());
 
     StreamManager streamManager = mock(StreamManager.class);
     doReturn(streamManager).when(runner).getStreamManager();
@@ -84,7 +84,7 @@ public class TestLocalApplicationRunner {
     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());
+    doReturn(plan).when(runner).getExecutionPlan(any());
 
     CoordinationUtilsFactory coordinationUtilsFactory = mock(CoordinationUtilsFactory.class);
     JobCoordinatorConfig mockJcConfig = mock(JobCoordinatorConfig.class);
@@ -93,6 +93,7 @@ public class TestLocalApplicationRunner {
 
     try {
       runner.run(app);
+      runner.waitForFinish();
     } catch (Throwable t) {
       assertNotNull(t); //no jobs exception
     }
@@ -112,7 +113,6 @@ public class TestLocalApplicationRunner {
     LocalApplicationRunner runner = spy(localRunner);
 
     StreamApplication app = mock(StreamApplication.class);
-    doNothing().when(app).init(anyObject(), anyObject());
 
     StreamManager streamManager = mock(StreamManager.class);
     doReturn(streamManager).when(runner).getStreamManager();
@@ -120,7 +120,7 @@ public class TestLocalApplicationRunner {
     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());
+    doReturn(plan).when(runner).getExecutionPlan(any());
 
     CoordinationUtils coordinationUtils = mock(CoordinationUtils.class);
     CoordinationUtilsFactory coordinationUtilsFactory = mock(CoordinationUtilsFactory.class);
@@ -136,6 +136,7 @@ public class TestLocalApplicationRunner {
 
     try {
       runner.run(app);
+      runner.waitForFinish();
     } catch (Throwable t) {
       assertNotNull(t); //no jobs exception
     }
@@ -153,7 +154,7 @@ public class TestLocalApplicationRunner {
       throws Exception {
     final Map<String, String> config = new HashMap<>();
     config.put(ApplicationConfig.APP_PROCESSOR_ID_GENERATOR_CLASS, UUIDGenerator.class.getName());
-    config.put(TaskConfig.TASK_CLASS(), "org.apache.samza.test.processor.IdentityStreamTask");
+    config.put(TaskConfig.TASK_CLASS(), "org.apache.samza.task.IdentityStreamTask");
 
     LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(config));
 
@@ -170,7 +171,7 @@ public class TestLocalApplicationRunner {
       }).when(sp).start();
 
     LocalApplicationRunner spy = spy(runner);
-    doReturn(sp).when(spy).createStreamProcessor(anyObject(), anyObject(), captor.capture());
+    doReturn(sp).when(spy).createStreamProcessor(any(Config.class), captor.capture());
 
     spy.runTask();
 
@@ -184,13 +185,12 @@ public class TestLocalApplicationRunner {
     config.put(ApplicationConfig.APP_PROCESSOR_ID_GENERATOR_CLASS, UUIDGenerator.class.getName());
     LocalApplicationRunner runner = spy(new LocalApplicationRunner(new MapConfig(config)));
     StreamApplication app = mock(StreamApplication.class);
-    doNothing().when(app).init(anyObject(), anyObject());
 
     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());
+    doReturn(plan).when(runner).getExecutionPlan(any());
 
     StreamProcessor sp = mock(StreamProcessor.class);
     ArgumentCaptor<StreamProcessorLifecycleListener> captor =
@@ -207,6 +207,7 @@ public class TestLocalApplicationRunner {
     doReturn(sp).when(runner).createStreamProcessor(anyObject(), anyObject(), captor.capture());
 
     runner.run(app);
+    runner.waitForFinish();
 
     assertEquals(runner.status(app), ApplicationStatus.SuccessfulFinish);
   }
@@ -218,13 +219,12 @@ public class TestLocalApplicationRunner {
     config.put(ApplicationConfig.PROCESSOR_ID, "0");
     LocalApplicationRunner runner = spy(new LocalApplicationRunner(new MapConfig(config)));
     StreamApplication app = mock(StreamApplication.class);
-    doNothing().when(app).init(anyObject(), anyObject());
 
     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());
+    doReturn(plan).when(runner).getExecutionPlan(any());
 
     StreamProcessor sp = mock(StreamProcessor.class);
     ArgumentCaptor<StreamProcessorLifecycleListener> captor =
@@ -239,6 +239,7 @@ public class TestLocalApplicationRunner {
 
     try {
       runner.run(app);
+      runner.waitForFinish();
     } catch (Throwable th) {
       assertNotNull(th);
     }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/task/IdentityStreamTask.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/task/IdentityStreamTask.java b/samza-core/src/test/java/org/apache/samza/task/IdentityStreamTask.java
new file mode 100644
index 0000000..1f71abd
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/task/IdentityStreamTask.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.task;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemStream;
+
+
+public class IdentityStreamTask implements StreamTask , InitableTask  {
+  private int processedMessageCount = 0;
+  private int expectedMessageCount;
+  private String outputTopic;
+  private String outputSystem;
+
+  @Override
+  public void init(Config config, TaskContext taskContext) throws Exception {
+    this.expectedMessageCount = config.getInt("app.messageCount");
+    this.outputTopic = config.get("app.outputTopic", "output");
+    this.outputSystem = config.get("app.outputSystem", "test-system");
+  }
+
+  @Override
+  public void process(
+      IncomingMessageEnvelope incomingMessageEnvelope,
+      MessageCollector messageCollector,
+      TaskCoordinator taskCoordinator) throws Exception {
+    messageCollector.send(
+        new OutgoingMessageEnvelope(
+            new SystemStream(outputSystem, outputTopic),
+            incomingMessageEnvelope.getMessage()));
+    processedMessageCount++;
+    if (processedMessageCount == expectedMessageCount) {
+      taskCoordinator.shutdown(TaskCoordinator.RequestScope.ALL_TASKS_IN_CONTAINER);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java
index 942792f..e207772 100644
--- a/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java
+++ b/samza-core/src/test/java/org/apache/samza/task/TestTaskFactoryUtil.java
@@ -18,20 +18,21 @@
  */
 package org.apache.samza.task;
 
+import java.lang.reflect.Field;
+import java.util.concurrent.ExecutorService;
 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.ConfigException;
 import org.apache.samza.config.MapConfig;
-import org.apache.samza.application.StreamApplication;
+import org.apache.samza.operators.StreamGraphSpec;
 import org.apache.samza.runtime.ApplicationRunner;
 import org.apache.samza.testUtils.TestAsyncStreamTask;
 import org.apache.samza.testUtils.TestStreamTask;
 import org.junit.Test;
 
-import java.lang.reflect.Field;
 import java.util.HashMap;
-import java.util.concurrent.ExecutorService;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -54,7 +55,7 @@ public class TestTaskFactoryUtil {
         this.put("task.class", "org.apache.samza.testUtils.TestStreamTask");
       }
     });
-    Object retFactory = TaskFactoryUtil.createTaskFactory(config, null, null);
+    Object retFactory = TaskFactoryUtil.createTaskFactory(config);
     assertTrue(retFactory instanceof StreamTaskFactory);
     assertTrue(((StreamTaskFactory) retFactory).createInstance() instanceof TestStreamTask);
 
@@ -64,7 +65,7 @@ public class TestTaskFactoryUtil {
       }
     });
     try {
-      TaskFactoryUtil.createTaskFactory(config, null, null);
+      TaskFactoryUtil.createTaskFactory(config);
       fail("Should have failed w/ no.such.class");
     } catch (ConfigException cfe) {
       // expected
@@ -80,7 +81,9 @@ public class TestTaskFactoryUtil {
     });
     StreamApplication streamApp = TaskFactoryUtil.createStreamApplication(config);
     assertNotNull(streamApp);
-    Object retFactory = TaskFactoryUtil.createTaskFactory(config, streamApp, mockRunner);
+    StreamGraphSpec graph = new StreamGraphSpec(mockRunner, config);
+    streamApp.init(graph, config);
+    Object retFactory = TaskFactoryUtil.createTaskFactory(graph.getOperatorSpecGraph(), null);
     assertTrue(retFactory instanceof StreamTaskFactory);
     assertTrue(((StreamTaskFactory) retFactory).createInstance() instanceof StreamOperatorTask);
 
@@ -184,29 +187,6 @@ public class TestTaskFactoryUtil {
       // expected
     }
 
-    config = new MapConfig(new HashMap<String, String>() {
-      {
-        this.put("task.class", "org.apache.samza.testUtils.TestStreamTask");
-        this.put(ApplicationConfig.APP_CLASS, "");
-      }
-    });
-    StreamApplication streamApp = TaskFactoryUtil.createStreamApplication(config);
-    Object retFactory = TaskFactoryUtil.createTaskFactory(config, streamApp, mockRunner);
-    assertTrue(retFactory instanceof StreamTaskFactory);
-    assertTrue(((StreamTaskFactory) retFactory).createInstance() instanceof TestStreamTask);
-
-    config = new MapConfig(new HashMap<String, String>() {
-      {
-        this.put("task.class", "");
-        this.put(ApplicationConfig.APP_CLASS, "org.apache.samza.testUtils.InvalidStreamApplication");
-      }
-    });
-    try {
-      TaskFactoryUtil.createStreamApplication(config);
-      fail("Should have failed w/ no class not found");
-    } catch (ConfigException cne) {
-      // expected
-    }
   }
 
   @Test
@@ -216,7 +196,7 @@ public class TestTaskFactoryUtil {
         this.put("task.class", "org.apache.samza.testUtils.TestAsyncStreamTask");
       }
     });
-    Object retFactory = TaskFactoryUtil.createTaskFactory(config, null, null);
+    Object retFactory = TaskFactoryUtil.createTaskFactory(config);
     assertTrue(retFactory instanceof AsyncStreamTaskFactory);
     assertTrue(((AsyncStreamTaskFactory) retFactory).createInstance() instanceof TestAsyncStreamTask);
 
@@ -226,7 +206,7 @@ public class TestTaskFactoryUtil {
       }
     });
     try {
-      TaskFactoryUtil.createTaskFactory(config, null, null);
+      TaskFactoryUtil.createTaskFactory(config);
       fail("Should have failed w/ no.such.class");
     } catch (ConfigException cfe) {
       // expected
@@ -247,28 +227,6 @@ public class TestTaskFactoryUtil {
     } catch (ConfigException cfe) {
       // expected
     }
-
-    config = new MapConfig(new HashMap<String, String>() {
-      {
-        this.put("task.class", "org.apache.samza.testUtils.TestAsyncStreamTask");
-        this.put(ApplicationConfig.APP_CLASS, "");
-      }
-    });
-    StreamApplication streamApp = TaskFactoryUtil.createStreamApplication(config);
-    Object retFactory = TaskFactoryUtil.createTaskFactory(config, streamApp, mockRunner);
-    assertTrue(retFactory instanceof AsyncStreamTaskFactory);
-    assertTrue(((AsyncStreamTaskFactory) retFactory).createInstance() instanceof TestAsyncStreamTask);
-
-    config = new MapConfig(new HashMap<String, String>() {
-      {
-        this.put("task.class", "org.apache.samza.testUtils.TestAsyncStreamTask");
-        this.put(ApplicationConfig.APP_CLASS, null);
-      }
-    });
-    streamApp = TaskFactoryUtil.createStreamApplication(config);
-    retFactory = TaskFactoryUtil.createTaskFactory(config, streamApp, mockRunner);
-    assertTrue(retFactory instanceof AsyncStreamTaskFactory);
-    assertTrue(((AsyncStreamTaskFactory) retFactory).createInstance() instanceof TestAsyncStreamTask);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/testUtils/InvalidStreamApplication.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/testUtils/InvalidStreamApplication.java b/samza-core/src/test/java/org/apache/samza/testUtils/InvalidStreamApplication.java
deleted file mode 100644
index 5026066..0000000
--- a/samza-core/src/test/java/org/apache/samza/testUtils/InvalidStreamApplication.java
+++ /dev/null
@@ -1,25 +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.testUtils;
-
-/**
- * Test class. Invalid class to implement {@link org.apache.samza.application.StreamApplication}
- */
-public class InvalidStreamApplication {
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 1758bf0..c00ed2d 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
@@ -21,13 +21,14 @@ package org.apache.samza.system.kafka;
 import com.google.common.collect.ImmutableMap;
 import java.util.Map;
 import java.util.Properties;
+import org.apache.samza.runtime.TestAbstractApplicationRunner;
 import org.apache.samza.system.StreamSpec;
 import org.junit.Test;
 
 import static org.junit.Assert.*;
 
 /**
- * See also the general StreamSpec tests in {@link org.apache.samza.runtime.TestAbstractApplicationRunner}
+ * See also the general StreamSpec tests in {@link TestAbstractApplicationRunner}
  */
 public class TestKafkaStreamSpec {
 

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/main/java/org/apache/samza/sql/data/SamzaSqlCompositeKey.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/data/SamzaSqlCompositeKey.java b/samza-sql/src/main/java/org/apache/samza/sql/data/SamzaSqlCompositeKey.java
index 54c8391..4b4b8f2 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/data/SamzaSqlCompositeKey.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/data/SamzaSqlCompositeKey.java
@@ -70,6 +70,7 @@ public class SamzaSqlCompositeKey implements Serializable {
    * Create the SamzaSqlCompositeKey from the rel message.
    * @param message Represents the samza sql rel message.
    * @param relIdx list of keys in the form of field indices within the rel message.
+   * @return the composite key of the rel message
    */
   public static SamzaSqlCompositeKey createSamzaSqlCompositeKey(SamzaSqlRelMessage message, List<Integer> relIdx) {
     ArrayList<Object> keyParts = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/main/java/org/apache/samza/sql/data/SamzaSqlExecutionContext.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/data/SamzaSqlExecutionContext.java b/samza-sql/src/main/java/org/apache/samza/sql/data/SamzaSqlExecutionContext.java
index b0c30dd..cb5c7a7 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/data/SamzaSqlExecutionContext.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/data/SamzaSqlExecutionContext.java
@@ -32,12 +32,24 @@ import org.apache.samza.sql.testutil.ReflectionUtils;
 import org.apache.samza.sql.udfs.ScalarUdf;
 
 
-public class SamzaSqlExecutionContext {
+public class SamzaSqlExecutionContext implements Cloneable {
 
+  /**
+   * The variables that are shared among all cloned instance of {@link SamzaSqlExecutionContext}
+   */
   private final SamzaSqlApplicationConfig sqlConfig;
   private final Map<String, UdfMetadata> udfMetadata;
+
+  /**
+   * The variable that are not shared among all cloned instance of {@link SamzaSqlExecutionContext}
+   */
   private final Map<String, ScalarUdf> udfInstances = new HashMap<>();
 
+  private SamzaSqlExecutionContext(SamzaSqlExecutionContext other) {
+    this.sqlConfig = other.sqlConfig;
+    this.udfMetadata = other.udfMetadata;
+  }
+
   public SamzaSqlExecutionContext(SamzaSqlApplicationConfig config) {
     this.sqlConfig = config;
     udfMetadata =
@@ -62,4 +74,10 @@ public class SamzaSqlExecutionContext {
   public SamzaSqlApplicationConfig getSamzaSqlApplicationConfig() {
     return sqlConfig;
   }
+
+  @Override
+  public SamzaSqlExecutionContext clone() {
+    return new SamzaSqlExecutionContext(this);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/main/java/org/apache/samza/sql/translator/FilterTranslator.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/FilterTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/FilterTranslator.java
index 5832b21..f33c5ca 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/translator/FilterTranslator.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/FilterTranslator.java
@@ -22,13 +22,13 @@ package org.apache.samza.sql.translator;
 import java.util.Arrays;
 import java.util.Collections;
 
-import java.util.List;
-import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.logical.LogicalFilter;
-import org.apache.calcite.rex.RexNode;
+import org.apache.samza.config.Config;
 import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.functions.FilterFunction;
 import org.apache.samza.sql.data.Expression;
 import org.apache.samza.sql.data.SamzaSqlRelMessage;
+import org.apache.samza.task.TaskContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -41,19 +41,26 @@ class FilterTranslator {
 
   private static final Logger log = LoggerFactory.getLogger(FilterTranslator.class);
 
-  void translate(final LogicalFilter filter, final TranslatorContext context) {
-    MessageStream<SamzaSqlRelMessage> inputStream = context.getMessageStream(filter.getInput().getId());
-    MessageStream<SamzaSqlRelMessage> outputStream = translateFilter(inputStream, filter.getInputs(),
-        filter.getCondition(), context);
-    context.registerMessageStream(filter.getId(), outputStream);
-  }
+  private static class FilterTranslatorFunction implements FilterFunction<SamzaSqlRelMessage> {
+    private transient Expression expr;
+    private transient TranslatorContext context;
+    private transient LogicalFilter filter;
 
-  static MessageStream<SamzaSqlRelMessage> translateFilter(MessageStream<SamzaSqlRelMessage> inputStream,
-      List<RelNode> inputs, RexNode condition, final TranslatorContext context) {
-    Expression expr =
-        context.getExpressionCompiler().compile(inputs, Collections.singletonList(condition));
+    private final int filterId;
 
-    return inputStream.filter(message -> {
+    FilterTranslatorFunction(int filterId) {
+      this.filterId = filterId;
+    }
+
+    @Override
+    public void init(Config config, TaskContext context) {
+      this.context = (TranslatorContext) context.getUserContext();
+      this.filter = (LogicalFilter) this.context.getRelNode(filterId);
+      this.expr = this.context.getExpressionCompiler().compile(filter.getInputs(), Collections.singletonList(filter.getCondition()));
+    }
+
+    @Override
+    public boolean apply(SamzaSqlRelMessage message) {
       Object[] result = new Object[1];
       expr.execute(context.getExecutionContext(), context.getDataContext(),
           message.getSamzaSqlRelRecord().getFieldValues().toArray(), result);
@@ -67,6 +74,16 @@ class FilterTranslator {
         log.error("return value is not boolean");
         return false;
       }
-    });
+    }
+  }
+
+  void translate(final LogicalFilter filter, final TranslatorContext context) {
+    MessageStream<SamzaSqlRelMessage> inputStream = context.getMessageStream(filter.getInput().getId());
+    final int filterId = filter.getId();
+
+    MessageStream<SamzaSqlRelMessage> outputStream = inputStream.filter(new FilterTranslatorFunction(filterId));
+
+    context.registerMessageStream(filterId, outputStream);
+    context.registerRelNode(filterId, filter);
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/main/java/org/apache/samza/sql/translator/LogicalAggregateTranslator.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/LogicalAggregateTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/LogicalAggregateTranslator.java
index 96179d6..216ecea 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/translator/LogicalAggregateTranslator.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/LogicalAggregateTranslator.java
@@ -20,13 +20,14 @@
 package org.apache.samza.sql.translator;
 
 import java.time.Duration;
+import java.util.ArrayList;
 import java.util.List;
-import java.util.function.Supplier;
 import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.samza.SamzaException;
 import org.apache.samza.operators.MessageStream;
 import org.apache.samza.operators.functions.FoldLeftFunction;
+import org.apache.samza.operators.functions.SupplierFunction;
 import org.apache.samza.operators.windows.AccumulationMode;
 import org.apache.samza.operators.windows.Windows;
 import org.apache.samza.serializers.LongSerde;
@@ -55,9 +56,11 @@ class LogicalAggregateTranslator {
     MessageStream<SamzaSqlRelMessage> inputStream = context.getMessageStream(aggregate.getInput().getId());
 
     // At this point, the assumption is that only count function is supported.
-    Supplier<Long> initialValue = () -> (long) 0;
+    SupplierFunction<Long> initialValue = () -> (long) 0;
     FoldLeftFunction<SamzaSqlRelMessage, Long> foldCountFn = (m, c) -> c + 1;
 
+    final ArrayList<String> aggFieldNames = getAggFieldNames(aggregate);
+
     MessageStream<SamzaSqlRelMessage> outputStream =
         inputStream
             .window(Windows.keyedTumblingWindow(m -> m,
@@ -68,15 +71,20 @@ class LogicalAggregateTranslator {
                 new LongSerde())
                 .setAccumulationMode(AccumulationMode.DISCARDING), "tumblingWindow_" + windowId)
             .map(windowPane -> {
-              List<String> fieldNames = windowPane.getKey().getKey().getSamzaSqlRelRecord().getFieldNames();
-              List<Object> fieldValues = windowPane.getKey().getKey().getSamzaSqlRelRecord().getFieldValues();
-              fieldNames.add(aggregate.getAggCallList().get(0).getName());
-              fieldValues.add(windowPane.getMessage());
-              return new SamzaSqlRelMessage(fieldNames, fieldValues);
-            });
+                List<String> fieldNames = windowPane.getKey().getKey().getSamzaSqlRelRecord().getFieldNames();
+                List<Object> fieldValues = windowPane.getKey().getKey().getSamzaSqlRelRecord().getFieldValues();
+                fieldNames.add(aggFieldNames.get(0));
+                fieldValues.add(windowPane.getMessage());
+                return new SamzaSqlRelMessage(fieldNames, fieldValues);
+              });
     context.registerMessageStream(aggregate.getId(), outputStream);
   }
 
+  private ArrayList<String> getAggFieldNames(LogicalAggregate aggregate) {
+    return aggregate.getAggCallList().stream().collect(ArrayList::new, (names, aggCall) -> names.add(aggCall.getName()),
+        (n1, n2) -> n1.addAll(n2));
+  }
+
   void validateAggregateFunctions(final LogicalAggregate aggregate) {
     if (aggregate.getAggCallList().size() != 1) {
       String errMsg = "Windowing is supported ONLY with one aggregate function but the number of given functions are " +


[09/10] samza git commit: SAMZA-1659: Serializable OperatorSpec

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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
deleted file mode 100644
index 7ddcd19..0000000
--- a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
+++ /dev/null
@@ -1,328 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.regex.Pattern;
-import java.util.stream.Collectors;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.samza.SamzaException;
-import org.apache.samza.config.Config;
-import org.apache.samza.config.JobConfig;
-import org.apache.samza.operators.spec.InputOperatorSpec;
-import org.apache.samza.operators.spec.OperatorSpec;
-import org.apache.samza.operators.spec.OperatorSpec.OpCode;
-import org.apache.samza.operators.spec.OperatorSpecs;
-import org.apache.samza.operators.spec.OutputStreamImpl;
-import org.apache.samza.operators.stream.IntermediateMessageStreamImpl;
-import org.apache.samza.runtime.ApplicationRunner;
-import org.apache.samza.serializers.KVSerde;
-import org.apache.samza.serializers.NoOpSerde;
-import org.apache.samza.serializers.Serde;
-import org.apache.samza.system.StreamSpec;
-import org.apache.samza.table.Table;
-import org.apache.samza.table.TableSpec;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Preconditions;
-
-/**
- * A {@link StreamGraph} that provides APIs for accessing {@link MessageStream}s to be used to
- * create the DAG of transforms.
- */
-public class StreamGraphImpl implements StreamGraph {
-  private static final Logger LOGGER = LoggerFactory.getLogger(StreamGraphImpl.class);
-  private static final Pattern USER_DEFINED_ID_PATTERN = Pattern.compile("[\\d\\w-_.]+");
-
-  // We use a LHM for deterministic order in initializing and closing operators.
-  private final Map<StreamSpec, InputOperatorSpec> inputOperators = new LinkedHashMap<>();
-  private final Map<StreamSpec, OutputStreamImpl> outputStreams = new LinkedHashMap<>();
-  private final Map<TableSpec, TableImpl> tables = new LinkedHashMap<>();
-  private final ApplicationRunner runner;
-  private final Config config;
-
-
-  /**
-   * The 0-based position of the next operator in the graph.
-   * Part of the unique ID for each OperatorSpec in the graph.
-   * Should only accessed and incremented via {@link #getNextOpId(OpCode, String)}.
-   */
-  private int nextOpNum = 0;
-  private final Set<String> operatorIds = new HashSet<>();
-  private Serde<?> defaultSerde = new KVSerde(new NoOpSerde(), new NoOpSerde());
-  private ContextManager contextManager = null;
-
-  public StreamGraphImpl(ApplicationRunner runner, Config config) {
-    // TODO: SAMZA-1118 - Move StreamSpec and ApplicationRunner out of StreamGraphImpl once Systems
-    // can use streamId to send and receive messages.
-    this.runner = runner;
-    this.config = config;
-  }
-
-  @Override
-  public void setDefaultSerde(Serde<?> serde) {
-    Preconditions.checkNotNull(serde, "Default serde must not be null");
-    Preconditions.checkState(inputOperators.isEmpty() && outputStreams.isEmpty(),
-        "Default serde must be set before creating any input or output streams.");
-    this.defaultSerde = serde;
-  }
-
-  @Override
-  public <M> MessageStream<M> getInputStream(String streamId, Serde<M> serde) {
-    StreamSpec streamSpec = runner.getStreamSpec(streamId);
-    Preconditions.checkState(streamSpec != null, "No StreamSpec found for streamId: " + streamId);
-    Preconditions.checkNotNull(serde, "serde must not be null for an input stream.");
-    Preconditions.checkState(!inputOperators.containsKey(streamSpec),
-        "getInputStream must not be called multiple times with the same streamId: " + streamId);
-
-    KV<Serde, Serde> kvSerdes = getKVSerdes(streamId, serde);
-    if (outputStreams.containsKey(streamSpec)) {
-      OutputStreamImpl outputStream = outputStreams.get(streamSpec);
-      Serde keySerde = outputStream.getKeySerde();
-      Serde valueSerde = outputStream.getValueSerde();
-      Preconditions.checkState(kvSerdes.getKey().equals(keySerde) && kvSerdes.getValue().equals(valueSerde),
-          String.format("Stream %s is being used both as an input and an output stream. Serde in Samza happens at "
-              + "stream level, so the same key and message Serde must be used for both.", streamId));
-    }
-
-    boolean isKeyed = serde instanceof KVSerde;
-    InputOperatorSpec inputOperatorSpec =
-        OperatorSpecs.createInputOperatorSpec(streamSpec, kvSerdes.getKey(), kvSerdes.getValue(),
-            isKeyed, this.getNextOpId(OpCode.INPUT, null));
-    inputOperators.put(streamSpec, inputOperatorSpec);
-    return new MessageStreamImpl<>(this, inputOperators.get(streamSpec));
-  }
-
-  @Override
-  public <M> MessageStream<M> getInputStream(String streamId) {
-    return (MessageStream<M>) getInputStream(streamId, defaultSerde);
-  }
-
-  @Override
-  public <M> OutputStream<M> getOutputStream(String streamId, Serde<M> serde) {
-    StreamSpec streamSpec = runner.getStreamSpec(streamId);
-    Preconditions.checkState(streamSpec != null, "No StreamSpec found for streamId: " + streamId);
-    Preconditions.checkNotNull(serde, "serde must not be null for an output stream.");
-    Preconditions.checkState(!outputStreams.containsKey(streamSpec),
-        "getOutputStream must not be called multiple times with the same streamId: " + streamId);
-
-    KV<Serde, Serde> kvSerdes = getKVSerdes(streamId, serde);
-    if (inputOperators.containsKey(streamSpec)) {
-      InputOperatorSpec inputOperatorSpec = inputOperators.get(streamSpec);
-      Serde keySerde = inputOperatorSpec.getKeySerde();
-      Serde valueSerde = inputOperatorSpec.getValueSerde();
-      Preconditions.checkState(kvSerdes.getKey().equals(keySerde) && kvSerdes.getValue().equals(valueSerde),
-          String.format("Stream %s is being used both as an input and an output stream. Serde in Samza happens at "
-              + "stream level, so the same key and message Serde must be used for both.", streamId));
-    }
-
-    boolean isKeyed = serde instanceof KVSerde;
-    outputStreams.put(streamSpec, new OutputStreamImpl<>(streamSpec, kvSerdes.getKey(), kvSerdes.getValue(), isKeyed));
-    return outputStreams.get(streamSpec);
-  }
-
-  @Override
-  public <M> OutputStream<M> getOutputStream(String streamId) {
-    return (OutputStream<M>) getOutputStream(streamId, defaultSerde);
-  }
-
-  @Override
-  public <K, V> Table<KV<K, V>> getTable(TableDescriptor<K, V, ?> tableDesc) {
-    TableSpec tableSpec = ((BaseTableDescriptor) tableDesc).getTableSpec();
-    if (tables.containsKey(tableSpec)) {
-      throw new IllegalStateException(String.format(
-          "getTable() invoked multiple times with the same tableId: %s",
-          tableDesc.getTableId()));
-    }
-    tables.put(tableSpec, new TableImpl(tableSpec));
-    return tables.get(tableSpec);
-  }
-
-  @Override
-  public StreamGraph withContextManager(ContextManager contextManager) {
-    this.contextManager = contextManager;
-    return this;
-  }
-
-  /**
-   * See {@link StreamGraphImpl#getIntermediateStream(String, Serde, boolean)}.
-   */
-  <M> IntermediateMessageStreamImpl<M> getIntermediateStream(String streamId, Serde<M> serde) {
-    return getIntermediateStream(streamId, serde, false);
-  }
-
-  /**
-   * Internal helper for {@link MessageStreamImpl} to add an intermediate {@link MessageStream} to the graph.
-   * An intermediate {@link MessageStream} is both an output and an input stream.
-   *
-   * @param streamId the id of the stream to be created.
-   * @param serde the {@link Serde} to use for the message in the intermediate stream. If null, the default serde
-   *              is used.
-   * @param isBroadcast whether the stream is a broadcast stream.
-   * @param <M> the type of messages in the intermediate {@link MessageStream}
-   * @return  the intermediate {@link MessageStreamImpl}
-   *
-   * TODO: once SAMZA-1566 is resolved, we should be able to pass in the StreamSpec directly.
-   */
-  <M> IntermediateMessageStreamImpl<M> getIntermediateStream(String streamId, Serde<M> serde, boolean isBroadcast) {
-    StreamSpec streamSpec = runner.getStreamSpec(streamId);
-    if (isBroadcast) {
-      streamSpec = streamSpec.copyWithBroadCast();
-    }
-
-    Preconditions.checkState(!inputOperators.containsKey(streamSpec) && !outputStreams.containsKey(streamSpec),
-        "getIntermediateStream must not be called multiple times with the same streamId: " + streamId);
-
-    if (serde == null) {
-      LOGGER.info("Using default serde for intermediate stream: " + streamId);
-      serde = (Serde<M>) defaultSerde;
-    }
-
-    boolean isKeyed = serde instanceof KVSerde;
-    KV<Serde, Serde> kvSerdes = getKVSerdes(streamId, serde);
-    InputOperatorSpec inputOperatorSpec =
-        OperatorSpecs.createInputOperatorSpec(streamSpec, kvSerdes.getKey(), kvSerdes.getValue(),
-            isKeyed, this.getNextOpId(OpCode.INPUT, null));
-    inputOperators.put(streamSpec, inputOperatorSpec);
-    outputStreams.put(streamSpec, new OutputStreamImpl(streamSpec, kvSerdes.getKey(), kvSerdes.getValue(), isKeyed));
-    return new IntermediateMessageStreamImpl<>(this, inputOperators.get(streamSpec), outputStreams.get(streamSpec));
-  }
-
-  public Map<StreamSpec, InputOperatorSpec> getInputOperators() {
-    return Collections.unmodifiableMap(inputOperators);
-  }
-
-  public Map<StreamSpec, OutputStreamImpl> getOutputStreams() {
-    return Collections.unmodifiableMap(outputStreams);
-  }
-
-  public Map<TableSpec, TableImpl> getTables() {
-    return Collections.unmodifiableMap(tables);
-  }
-
-  public ContextManager getContextManager() {
-    return this.contextManager;
-  }
-
-  /**
-   * Gets the unique ID for the next operator in the graph. The ID is of the following format:
-   * jobName-jobId-opCode-(userDefinedId|nextOpNum);
-   *
-   * @param opCode the {@link OpCode} of the next operator
-   * @param userDefinedId the optional user-provided name of the next operator or null
-   * @return the unique ID for the next operator in the graph
-   */
-  /* package private */ String getNextOpId(OpCode opCode, String userDefinedId) {
-    if (StringUtils.isNotBlank(userDefinedId) && !USER_DEFINED_ID_PATTERN.matcher(userDefinedId).matches()) {
-      throw new SamzaException("Operator ID must not contain spaces and special characters: " + userDefinedId);
-    }
-
-    String nextOpId = String.format("%s-%s-%s-%s",
-        config.get(JobConfig.JOB_NAME()),
-        config.get(JobConfig.JOB_ID(), "1"),
-        opCode.name().toLowerCase(),
-        StringUtils.isNotBlank(userDefinedId) ? userDefinedId.trim() : String.valueOf(nextOpNum));
-    if (!operatorIds.add(nextOpId)) {
-      throw new SamzaException(
-          String.format("Found duplicate operator ID %s in the graph. Operator IDs must be unique.", nextOpId));
-    }
-    nextOpNum++;
-    return nextOpId;
-  }
-
-  /**
-   * Gets the unique ID for the next operator in the graph. The ID is of the following format:
-   * jobName-jobId-opCode-nextOpNum;
-   *
-   * @param opCode the {@link OpCode} of the next operator
-   * @return the unique ID for the next operator in the graph
-   */
-  /* package private */ String getNextOpId(OpCode opCode) {
-    return getNextOpId(opCode, null);
-  }
-
-  /**
-   * Get all {@link OperatorSpec}s available in this {@link StreamGraphImpl}
-   *
-   * @return  all available {@link OperatorSpec}s
-   */
-  public Collection<OperatorSpec> getAllOperatorSpecs() {
-    Collection<InputOperatorSpec> inputOperatorSpecs = inputOperators.values();
-    Set<OperatorSpec> operatorSpecs = new HashSet<>();
-    for (InputOperatorSpec inputOperatorSpec: inputOperatorSpecs) {
-      operatorSpecs.add(inputOperatorSpec);
-      doGetOperatorSpecs(inputOperatorSpec, operatorSpecs);
-    }
-    return operatorSpecs;
-  }
-
-  private void doGetOperatorSpecs(OperatorSpec operatorSpec, Set<OperatorSpec> specs) {
-    Collection<OperatorSpec> registeredOperatorSpecs = operatorSpec.getRegisteredOperatorSpecs();
-    for (OperatorSpec registeredOperatorSpec: registeredOperatorSpecs) {
-      specs.add(registeredOperatorSpec);
-      doGetOperatorSpecs(registeredOperatorSpec, specs);
-    }
-  }
-
-  /**
-   * Returns <tt>true</tt> iff this {@link StreamGraphImpl} contains a join or a window operator
-   *
-   * @return  <tt>true</tt> iff this {@link StreamGraphImpl} contains a join or a window operator
-   */
-  public boolean hasWindowOrJoins() {
-    // Obtain the operator specs from the streamGraph
-    Collection<OperatorSpec> operatorSpecs = getAllOperatorSpecs();
-
-    Set<OperatorSpec> windowOrJoinSpecs = operatorSpecs.stream()
-        .filter(spec -> spec.getOpCode() == OperatorSpec.OpCode.WINDOW || spec.getOpCode() == OperatorSpec.OpCode.JOIN)
-        .collect(Collectors.toSet());
-
-    return windowOrJoinSpecs.size() != 0;
-  }
-
-  private KV<Serde, Serde> getKVSerdes(String streamId, Serde serde) {
-    Serde keySerde, valueSerde;
-
-    if (serde instanceof KVSerde) {
-      keySerde = ((KVSerde) serde).getKeySerde();
-      valueSerde = ((KVSerde) serde).getValueSerde();
-    } else {
-      keySerde = new NoOpSerde();
-      valueSerde = serde;
-    }
-
-    if (keySerde instanceof NoOpSerde) {
-      LOGGER.info("Using NoOpSerde as the key serde for stream " + streamId +
-          ". Keys will not be (de)serialized");
-    }
-    if (valueSerde instanceof NoOpSerde) {
-      LOGGER.info("Using NoOpSerde as the value serde for stream " + streamId +
-          ". Values will not be (de)serialized");
-    }
-
-    return KV.of(keySerde, valueSerde);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java
new file mode 100644
index 0000000..ea9690b
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphSpec.java
@@ -0,0 +1,299 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.operators.spec.InputOperatorSpec;
+import org.apache.samza.operators.spec.OperatorSpec.OpCode;
+import org.apache.samza.operators.spec.OperatorSpecs;
+import org.apache.samza.operators.spec.OutputStreamImpl;
+import org.apache.samza.operators.stream.IntermediateMessageStreamImpl;
+import org.apache.samza.runtime.ApplicationRunner;
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.serializers.NoOpSerde;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.system.StreamSpec;
+import org.apache.samza.table.Table;
+import org.apache.samza.table.TableSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+/**
+ * This class defines:
+ * 1) an implementation of {@link StreamGraph} that provides APIs for accessing {@link MessageStream}s to be used to
+ * create the DAG of transforms.
+ * 2) a builder that creates a serializable {@link OperatorSpecGraph} from user-defined DAG
+ */
+public class StreamGraphSpec implements StreamGraph {
+  private static final Logger LOGGER = LoggerFactory.getLogger(StreamGraphSpec.class);
+  private static final Pattern USER_DEFINED_ID_PATTERN = Pattern.compile("[\\d\\w-_.]+");
+
+  // We use a LHM for deterministic order in initializing and closing operators.
+  private final Map<StreamSpec, InputOperatorSpec> inputOperators = new LinkedHashMap<>();
+  private final Map<StreamSpec, OutputStreamImpl> outputStreams = new LinkedHashMap<>();
+  private final Map<TableSpec, TableImpl> tables = new LinkedHashMap<>();
+  private final ApplicationRunner runner;
+  private final Config config;
+
+  /**
+   * The 0-based position of the next operator in the graph.
+   * Part of the unique ID for each OperatorSpec in the graph.
+   * Should only accessed and incremented via {@link #getNextOpId(OpCode, String)}.
+   */
+  private int nextOpNum = 0;
+  private final Set<String> operatorIds = new HashSet<>();
+  private Serde<?> defaultSerde = new KVSerde(new NoOpSerde(), new NoOpSerde());
+  private ContextManager contextManager = null;
+
+  public StreamGraphSpec(ApplicationRunner runner, Config config) {
+    // TODO: SAMZA-1118 - Move StreamSpec and ApplicationRunner out of StreamGraphSpec once Systems
+    // can use streamId to send and receive messages.
+    this.runner = runner;
+    this.config = config;
+  }
+
+  @Override
+  public void setDefaultSerde(Serde<?> serde) {
+    Preconditions.checkNotNull(serde, "Default serde must not be null");
+    Preconditions.checkState(inputOperators.isEmpty() && outputStreams.isEmpty(),
+        "Default serde must be set before creating any input or output streams.");
+    this.defaultSerde = serde;
+  }
+
+  @Override
+  public <M> MessageStream<M> getInputStream(String streamId, Serde<M> serde) {
+    StreamSpec streamSpec = runner.getStreamSpec(streamId);
+    Preconditions.checkState(streamSpec != null, "No StreamSpec found for streamId: " + streamId);
+    Preconditions.checkNotNull(serde, "serde must not be null for an input stream.");
+    Preconditions.checkState(!inputOperators.containsKey(streamSpec),
+        "getInputStream must not be called multiple times with the same streamId: " + streamId);
+
+    KV<Serde, Serde> kvSerdes = getKVSerdes(streamId, serde);
+    if (outputStreams.containsKey(streamSpec)) {
+      OutputStreamImpl outputStream = outputStreams.get(streamSpec);
+      Serde keySerde = outputStream.getKeySerde();
+      Serde valueSerde = outputStream.getValueSerde();
+      Preconditions.checkState(kvSerdes.getKey().equals(keySerde) && kvSerdes.getValue().equals(valueSerde),
+          String.format("Stream %s is being used both as an input and an output stream. Serde in Samza happens at "
+              + "stream level, so the same key and message Serde must be used for both.", streamId));
+    }
+
+    boolean isKeyed = serde instanceof KVSerde;
+    InputOperatorSpec inputOperatorSpec =
+        OperatorSpecs.createInputOperatorSpec(streamSpec, kvSerdes.getKey(), kvSerdes.getValue(),
+            isKeyed, this.getNextOpId(OpCode.INPUT, null));
+    inputOperators.put(streamSpec, inputOperatorSpec);
+    return new MessageStreamImpl<>(this, inputOperators.get(streamSpec));
+  }
+
+  @Override
+  public <M> MessageStream<M> getInputStream(String streamId) {
+    return (MessageStream<M>) getInputStream(streamId, defaultSerde);
+  }
+
+  @Override
+  public <M> OutputStream<M> getOutputStream(String streamId, Serde<M> serde) {
+    StreamSpec streamSpec = runner.getStreamSpec(streamId);
+    Preconditions.checkState(streamSpec != null, "No StreamSpec found for streamId: " + streamId);
+    Preconditions.checkNotNull(serde, "serde must not be null for an output stream.");
+    Preconditions.checkState(!outputStreams.containsKey(streamSpec),
+        "getOutputStream must not be called multiple times with the same streamId: " + streamId);
+
+    KV<Serde, Serde> kvSerdes = getKVSerdes(streamId, serde);
+    if (inputOperators.containsKey(streamSpec)) {
+      InputOperatorSpec inputOperatorSpec = inputOperators.get(streamSpec);
+      Serde keySerde = inputOperatorSpec.getKeySerde();
+      Serde valueSerde = inputOperatorSpec.getValueSerde();
+      Preconditions.checkState(kvSerdes.getKey().equals(keySerde) && kvSerdes.getValue().equals(valueSerde),
+          String.format("Stream %s is being used both as an input and an output stream. Serde in Samza happens at "
+              + "stream level, so the same key and message Serde must be used for both.", streamId));
+    }
+
+    boolean isKeyed = serde instanceof KVSerde;
+    outputStreams.put(streamSpec, new OutputStreamImpl<>(streamSpec, kvSerdes.getKey(), kvSerdes.getValue(), isKeyed));
+    return outputStreams.get(streamSpec);
+  }
+
+  @Override
+  public <M> OutputStream<M> getOutputStream(String streamId) {
+    return (OutputStream<M>) getOutputStream(streamId, defaultSerde);
+  }
+
+  @Override
+  public <K, V> Table<KV<K, V>> getTable(TableDescriptor<K, V, ?> tableDesc) {
+    TableSpec tableSpec = ((BaseTableDescriptor) tableDesc).getTableSpec();
+    if (tables.containsKey(tableSpec)) {
+      throw new IllegalStateException(String.format(
+          "getTable() invoked multiple times with the same tableId: %s",
+          tableDesc.getTableId()));
+    }
+    tables.put(tableSpec, new TableImpl(tableSpec));
+    return tables.get(tableSpec);
+  }
+
+  @Override
+  public StreamGraph withContextManager(ContextManager contextManager) {
+    this.contextManager = contextManager;
+    return this;
+  }
+
+  public ContextManager getContextManager() {
+    return this.contextManager;
+  }
+
+  public OperatorSpecGraph getOperatorSpecGraph() {
+    return new OperatorSpecGraph(this);
+  }
+
+  /**
+   * Gets the unique ID for the next operator in the graph. The ID is of the following format:
+   * jobName-jobId-opCode-(userDefinedId|nextOpNum);
+   *
+   * @param opCode the {@link OpCode} of the next operator
+   * @param userDefinedId the optional user-provided name of the next operator or null
+   * @return the unique ID for the next operator in the graph
+   */
+  public String getNextOpId(OpCode opCode, String userDefinedId) {
+    if (StringUtils.isNotBlank(userDefinedId) && !USER_DEFINED_ID_PATTERN.matcher(userDefinedId).matches()) {
+      throw new SamzaException("Operator ID must not contain spaces and special characters: " + userDefinedId);
+    }
+
+    String nextOpId = String.format("%s-%s-%s-%s",
+        config.get(JobConfig.JOB_NAME()),
+        config.get(JobConfig.JOB_ID(), "1"),
+        opCode.name().toLowerCase(),
+        StringUtils.isNotBlank(userDefinedId) ? userDefinedId.trim() : String.valueOf(nextOpNum));
+    if (!operatorIds.add(nextOpId)) {
+      throw new SamzaException(
+          String.format("Found duplicate operator ID %s in the graph. Operator IDs must be unique.", nextOpId));
+    }
+    nextOpNum++;
+    return nextOpId;
+  }
+
+  /**
+   * Gets the unique ID for the next operator in the graph. The ID is of the following format:
+   * jobName-jobId-opCode-nextOpNum;
+   *
+   * @param opCode the {@link OpCode} of the next operator
+   * @return the unique ID for the next operator in the graph
+   */
+  public String getNextOpId(OpCode opCode) {
+    return getNextOpId(opCode, null);
+  }
+
+  /**
+   * See {@link StreamGraphSpec#getIntermediateStream(String, Serde, boolean)}.
+   *
+   * @param <M> type of messages in the intermediate stream
+   * @param streamId the id of the stream to be created
+   * @param serde the {@link Serde} to use for messages in the intermediate stream. If null, the default serde is used.
+   * @return  the intermediate {@link MessageStreamImpl}
+   */
+  @VisibleForTesting
+  public <M> IntermediateMessageStreamImpl<M> getIntermediateStream(String streamId, Serde<M> serde) {
+    return getIntermediateStream(streamId, serde, false);
+  }
+
+  /**
+   * Internal helper for {@link MessageStreamImpl} to add an intermediate {@link MessageStream} to the graph.
+   * An intermediate {@link MessageStream} is both an output and an input stream.
+   *
+   * @param streamId the id of the stream to be created.
+   * @param serde the {@link Serde} to use for the message in the intermediate stream. If null, the default serde
+   *              is used.
+   * @param isBroadcast whether the stream is a broadcast stream.
+   * @param <M> the type of messages in the intermediate {@link MessageStream}
+   * @return  the intermediate {@link MessageStreamImpl}
+   *
+   * TODO: once SAMZA-1566 is resolved, we should be able to pass in the StreamSpec directly.
+   */
+  @VisibleForTesting
+  <M> IntermediateMessageStreamImpl<M> getIntermediateStream(String streamId, Serde<M> serde, boolean isBroadcast) {
+    StreamSpec streamSpec = runner.getStreamSpec(streamId);
+    if (isBroadcast) {
+      streamSpec = streamSpec.copyWithBroadCast();
+    }
+
+    Preconditions.checkState(!inputOperators.containsKey(streamSpec) && !outputStreams.containsKey(streamSpec),
+        "getIntermediateStream must not be called multiple times with the same streamId: " + streamId);
+
+    if (serde == null) {
+      LOGGER.info("Using default serde for intermediate stream: " + streamId);
+      serde = (Serde<M>) defaultSerde;
+    }
+
+    boolean isKeyed = serde instanceof KVSerde;
+    KV<Serde, Serde> kvSerdes = getKVSerdes(streamId, serde);
+    InputOperatorSpec inputOperatorSpec =
+        OperatorSpecs.createInputOperatorSpec(streamSpec, kvSerdes.getKey(), kvSerdes.getValue(),
+            isKeyed, this.getNextOpId(OpCode.INPUT, null));
+    inputOperators.put(streamSpec, inputOperatorSpec);
+    outputStreams.put(streamSpec, new OutputStreamImpl(streamSpec, kvSerdes.getKey(), kvSerdes.getValue(), isKeyed));
+    return new IntermediateMessageStreamImpl<>(this, inputOperators.get(streamSpec), outputStreams.get(streamSpec));
+  }
+
+  Map<StreamSpec, InputOperatorSpec> getInputOperators() {
+    return Collections.unmodifiableMap(inputOperators);
+  }
+
+  Map<StreamSpec, OutputStreamImpl> getOutputStreams() {
+    return Collections.unmodifiableMap(outputStreams);
+  }
+
+  Map<TableSpec, TableImpl> getTables() {
+    return Collections.unmodifiableMap(tables);
+  }
+
+  private KV<Serde, Serde> getKVSerdes(String streamId, Serde serde) {
+    Serde keySerde, valueSerde;
+
+    if (serde instanceof KVSerde) {
+      keySerde = ((KVSerde) serde).getKeySerde();
+      valueSerde = ((KVSerde) serde).getValueSerde();
+    } else {
+      keySerde = new NoOpSerde();
+      valueSerde = serde;
+    }
+
+    if (keySerde instanceof NoOpSerde) {
+      LOGGER.info("Using NoOpSerde as the key serde for stream " + streamId +
+          ". Keys will not be (de)serialized");
+    }
+    if (valueSerde instanceof NoOpSerde) {
+      LOGGER.info("Using NoOpSerde as the value serde for stream " + streamId +
+          ". Values will not be (de)serialized");
+    }
+
+    return KV.of(keySerde, valueSerde);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/main/java/org/apache/samza/operators/TableImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/TableImpl.java b/samza-core/src/main/java/org/apache/samza/operators/TableImpl.java
index e671534..8ceada0 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/TableImpl.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/TableImpl.java
@@ -18,6 +18,7 @@
  */
 package org.apache.samza.operators;
 
+import java.io.Serializable;
 import org.apache.samza.table.Table;
 import org.apache.samza.table.TableSpec;
 
@@ -25,7 +26,7 @@ import org.apache.samza.table.TableSpec;
 /**
  * This class is the holder of a {@link TableSpec}
  */
-public class TableImpl implements Table {
+public class TableImpl implements Table, Serializable {
 
   private final TableSpec tableSpec;
 

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/main/java/org/apache/samza/operators/impl/BroadcastOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/BroadcastOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/BroadcastOperatorImpl.java
index 269e7bc..8df670e 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/impl/BroadcastOperatorImpl.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/BroadcastOperatorImpl.java
@@ -42,7 +42,7 @@ class BroadcastOperatorImpl<M> extends OperatorImpl<M, Void> {
 
   BroadcastOperatorImpl(BroadcastOperatorSpec<M> broadcastOpSpec, TaskContext context) {
     this.broadcastOpSpec = broadcastOpSpec;
-    this.systemStream = broadcastOpSpec.getOutputStream().getStreamSpec().toSystemStream();
+    this.systemStream = broadcastOpSpec.getOutputStream().getSystemStream();
     this.taskName = context.getTaskName().getTaskName();
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 608b2be..f0c0997 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
@@ -196,7 +196,7 @@ public abstract class OperatorImpl<M, RM> {
 
     results.forEach(rm ->
         this.registeredOperators.forEach(op ->
-            op.onMessage(rm, collector, coordinator)));    
+            op.onMessage(rm, collector, coordinator)));
 
     WatermarkFunction watermarkFn = getOperatorSpec().getWatermarkFn();
     if (watermarkFn != null) {

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 bbc8783..0f51798 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
@@ -25,7 +25,6 @@ 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.TimerRegistry;
 import org.apache.samza.operators.functions.JoinFunction;
 import org.apache.samza.operators.functions.PartialJoinFunction;
@@ -34,6 +33,7 @@ import org.apache.samza.operators.spec.BroadcastOperatorSpec;
 import org.apache.samza.operators.spec.InputOperatorSpec;
 import org.apache.samza.operators.spec.JoinOperatorSpec;
 import org.apache.samza.operators.spec.OperatorSpec;
+import org.apache.samza.operators.OperatorSpecGraph;
 import org.apache.samza.operators.spec.OutputOperatorSpec;
 import org.apache.samza.operators.spec.PartitionByOperatorSpec;
 import org.apache.samza.operators.spec.SendToTableOperatorSpec;
@@ -81,26 +81,26 @@ public class OperatorImplGraph {
    * the two {@link PartialJoinOperatorImpl}s for a {@link JoinOperatorSpec} with each other since they're
    * reached from different {@link OperatorSpec} during DAG traversals.
    */
-  private final Map<String, KV<PartialJoinFunction, PartialJoinFunction>> joinFunctions = new HashMap<>();
+  private final Map<String, KV<PartialJoinOperatorImpl, PartialJoinOperatorImpl>> joinOpImpls = new HashMap<>();
 
   private final Clock clock;
 
   /**
    * Constructs the DAG of {@link OperatorImpl}s corresponding to the the DAG of {@link OperatorSpec}s
-   * in the {@code streamGraph}.
+   * in the {@code specGraph}.
    *
-   * @param streamGraph  the {@link StreamGraphImpl} containing the logical {@link OperatorSpec} DAG
+   * @param specGraph  the {@link OperatorSpecGraph} containing the logical {@link OperatorSpec} DAG
    * @param config  the {@link Config} required to instantiate operators
    * @param context  the {@link TaskContext} required to instantiate operators
    * @param clock  the {@link Clock} to get current time
    */
-  public OperatorImplGraph(StreamGraphImpl streamGraph, Config config, TaskContext context, Clock clock) {
+  public OperatorImplGraph(OperatorSpecGraph specGraph, Config config, TaskContext context, Clock clock) {
     this.clock = clock;
 
     TaskContextImpl taskContext = (TaskContextImpl) context;
-    Map<SystemStream, Integer> producerTaskCounts = hasIntermediateStreams(streamGraph) ?
+    Map<SystemStream, Integer> producerTaskCounts = hasIntermediateStreams(specGraph) ?
         getProducerTaskCountForIntermediateStreams(getStreamToConsumerTasks(taskContext.getJobModel()),
-            getIntermediateToInputStreamsMap(streamGraph)) :
+            getIntermediateToInputStreamsMap(specGraph)) :
         Collections.EMPTY_MAP;
     producerTaskCounts.forEach((stream, count) -> {
         LOG.info("{} has {} producer tasks.", stream, count);
@@ -113,7 +113,7 @@ public class OperatorImplGraph {
     taskContext.registerObject(WatermarkStates.class.getName(),
         new WatermarkStates(context.getSystemStreamPartitions(), producerTaskCounts));
 
-    streamGraph.getInputOperators().forEach((streamSpec, inputOpSpec) -> {
+    specGraph.getInputOperators().forEach((streamSpec, inputOpSpec) -> {
         SystemStream systemStream = new SystemStream(streamSpec.getSystemName(), streamSpec.getPhysicalName());
         InputOperatorImpl inputOperatorImpl =
             (InputOperatorImpl) createAndRegisterOperatorImpl(null, inputOpSpec, systemStream, config, context);
@@ -151,12 +151,13 @@ public class OperatorImplGraph {
    * creates the corresponding DAG of {@link OperatorImpl}s, and returns the root {@link OperatorImpl} node.
    *
    * @param prevOperatorSpec  the parent of the current {@code operatorSpec} in the traversal
-   * @param operatorSpec  the operatorSpec to create the {@link OperatorImpl} for
+   * @param operatorSpec  the {@link OperatorSpec} to create the {@link OperatorImpl} for
+   * @param inputStream  the source input stream that we traverse the {@link OperatorSpecGraph} from
    * @param config  the {@link Config} required to instantiate operators
    * @param context  the {@link TaskContext} required to instantiate operators
    * @return  the operator implementation for the operatorSpec
    */
-  OperatorImpl createAndRegisterOperatorImpl(OperatorSpec prevOperatorSpec, OperatorSpec operatorSpec,
+  private OperatorImpl createAndRegisterOperatorImpl(OperatorSpec prevOperatorSpec, OperatorSpec operatorSpec,
       SystemStream inputStream, Config config, TaskContext context) {
 
     if (!operatorImpls.containsKey(operatorSpec.getOpId()) || operatorSpec instanceof JoinOperatorSpec) {
@@ -178,7 +179,9 @@ public class OperatorImplGraph {
 
       Collection<OperatorSpec> registeredSpecs = operatorSpec.getRegisteredOperatorSpecs();
       registeredSpecs.forEach(registeredSpec -> {
-          OperatorImpl nextImpl = createAndRegisterOperatorImpl(operatorSpec, registeredSpec, inputStream, config, context);
+          LOG.debug("Creating operator {} with opCode: {}", registeredSpec.getOpId(), registeredSpec.getOpCode());
+          OperatorImpl nextImpl =
+              createAndRegisterOperatorImpl(operatorSpec, registeredSpec, inputStream, config, context);
           operatorImpl.registerNextOperator(nextImpl);
         });
       return operatorImpl;
@@ -199,7 +202,8 @@ public class OperatorImplGraph {
   /**
    * Creates a new {@link OperatorImpl} instance for the provided {@link OperatorSpec}.
    *
-   * @param operatorSpec  the immutable {@link OperatorSpec} definition.
+   * @param prevOperatorSpec the original {@link OperatorSpec} that produces output for {@code operatorSpec} from {@link OperatorSpecGraph}
+   * @param operatorSpec  the original {@link OperatorSpec} from {@link OperatorSpecGraph}
    * @param config  the {@link Config} required to instantiate operators
    * @param context  the {@link TaskContext} required to instantiate operators
    * @return  the {@link OperatorImpl} implementation instance
@@ -209,17 +213,19 @@ public class OperatorImplGraph {
     if (operatorSpec instanceof InputOperatorSpec) {
       return new InputOperatorImpl((InputOperatorSpec) operatorSpec);
     } else if (operatorSpec instanceof StreamOperatorSpec) {
-      return new StreamOperatorImpl((StreamOperatorSpec) operatorSpec, config, context);
+      return new StreamOperatorImpl((StreamOperatorSpec) operatorSpec);
     } else if (operatorSpec instanceof SinkOperatorSpec) {
       return new SinkOperatorImpl((SinkOperatorSpec) operatorSpec, config, context);
     } else if (operatorSpec instanceof OutputOperatorSpec) {
-      return new OutputOperatorImpl((OutputOperatorSpec) operatorSpec, config, context);
+      return new OutputOperatorImpl((OutputOperatorSpec) operatorSpec);
     } else if (operatorSpec instanceof PartitionByOperatorSpec) {
       return new PartitionByOperatorImpl((PartitionByOperatorSpec) operatorSpec, config, context);
     } else if (operatorSpec instanceof WindowOperatorSpec) {
       return new WindowOperatorImpl((WindowOperatorSpec) operatorSpec, clock);
     } else if (operatorSpec instanceof JoinOperatorSpec) {
-      return createPartialJoinOperatorImpl(prevOperatorSpec, (JoinOperatorSpec) operatorSpec, config, context, clock);
+      return getOrCreatePartialJoinOpImpls((JoinOperatorSpec) operatorSpec,
+          prevOperatorSpec.equals(((JoinOperatorSpec) operatorSpec).getLeftInputOpSpec()),
+          config, context, clock);
     } else if (operatorSpec instanceof StreamTableJoinOperatorSpec) {
       return new StreamTableJoinOperatorImpl((StreamTableJoinOperatorSpec) operatorSpec, config, context);
     } else if (operatorSpec instanceof SendToTableOperatorSpec) {
@@ -231,23 +237,24 @@ public class OperatorImplGraph {
         String.format("Unsupported OperatorSpec: %s", operatorSpec.getClass().getName()));
   }
 
-  private PartialJoinOperatorImpl createPartialJoinOperatorImpl(OperatorSpec prevOperatorSpec,
-      JoinOperatorSpec joinOpSpec, Config config, TaskContext context, Clock clock) {
-    KV<PartialJoinFunction, PartialJoinFunction> partialJoinFunctions = getOrCreatePartialJoinFunctions(joinOpSpec);
-    if (joinOpSpec.getLeftInputOpSpec().equals(prevOperatorSpec)) { // we got here from the left side of the join
-      return new PartialJoinOperatorImpl(joinOpSpec, /* isLeftSide */ true,
-          partialJoinFunctions.getKey(), partialJoinFunctions.getValue(), config, context, clock);
+  private PartialJoinOperatorImpl getOrCreatePartialJoinOpImpls(JoinOperatorSpec joinOpSpec, boolean isLeft,
+      Config config, TaskContext context, Clock clock) {
+    // get the per task pair of PartialJoinOperatorImpl for the corresponding {@code joinOpSpec}
+    KV<PartialJoinOperatorImpl, PartialJoinOperatorImpl> partialJoinOpImpls = joinOpImpls.computeIfAbsent(joinOpSpec.getOpId(),
+        joinOpId -> {
+        PartialJoinFunction leftJoinFn = createLeftJoinFn(joinOpSpec);
+        PartialJoinFunction rightJoinFn = createRightJoinFn(joinOpSpec);
+        return new KV(new PartialJoinOperatorImpl(joinOpSpec, true, leftJoinFn, rightJoinFn, config, context, clock),
+            new PartialJoinOperatorImpl(joinOpSpec, false, rightJoinFn, leftJoinFn, config, context, clock));
+      });
+
+    if (isLeft) { // we got here from the left side of the join
+      return partialJoinOpImpls.getKey();
     } else { // we got here from the right side of the join
-      return new PartialJoinOperatorImpl(joinOpSpec, /* isLeftSide */ false,
-          partialJoinFunctions.getValue(), partialJoinFunctions.getKey(), config, context, clock);
+      return partialJoinOpImpls.getValue();
     }
   }
 
-  private KV<PartialJoinFunction, PartialJoinFunction> getOrCreatePartialJoinFunctions(JoinOperatorSpec joinOpSpec) {
-    return joinFunctions.computeIfAbsent(joinOpSpec.getOpId(),
-        joinOpId -> KV.of(createLeftJoinFn(joinOpSpec), createRightJoinFn(joinOpSpec)));
-  }
-
   private PartialJoinFunction<Object, Object, Object, Object> createLeftJoinFn(JoinOperatorSpec joinOpSpec) {
     return new PartialJoinFunction<Object, Object, Object, Object>() {
       private final JoinFunction joinFn = joinOpSpec.getJoinFn();
@@ -316,8 +323,8 @@ public class OperatorImplGraph {
     };
   }
 
-  private boolean hasIntermediateStreams(StreamGraphImpl streamGraph) {
-    return !Collections.disjoint(streamGraph.getInputOperators().keySet(), streamGraph.getOutputStreams().keySet());
+  private boolean hasIntermediateStreams(OperatorSpecGraph specGraph) {
+    return !Collections.disjoint(specGraph.getInputOperators().keySet(), specGraph.getOutputStreams().keySet());
   }
 
   /**
@@ -358,12 +365,12 @@ public class OperatorImplGraph {
 
   /**
    * calculate the mapping from output streams to input streams
-   * @param streamGraph the user {@link StreamGraphImpl} instance
+   * @param specGraph the user {@link OperatorSpecGraph}
    * @return mapping from output streams to input streams
    */
-  static Multimap<SystemStream, SystemStream> getIntermediateToInputStreamsMap(StreamGraphImpl streamGraph) {
+  static Multimap<SystemStream, SystemStream> getIntermediateToInputStreamsMap(OperatorSpecGraph specGraph) {
     Multimap<SystemStream, SystemStream> outputToInputStreams = HashMultimap.create();
-    streamGraph.getInputOperators().entrySet().stream()
+    specGraph.getInputOperators().entrySet().stream()
         .forEach(
             entry -> computeOutputToInput(entry.getKey().toSystemStream(), entry.getValue(), outputToInputStreams));
     return outputToInputStreams;

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 27bef87..e625484 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
@@ -42,11 +42,10 @@ class OutputOperatorImpl<M> extends OperatorImpl<M, Void> {
   private final OutputStreamImpl<M> outputStream;
   private final SystemStream systemStream;
 
-  OutputOperatorImpl(OutputOperatorSpec<M> outputOpSpec, Config config, TaskContext context) {
+  OutputOperatorImpl(OutputOperatorSpec<M> outputOpSpec) {
     this.outputOpSpec = outputOpSpec;
     this.outputStream = outputOpSpec.getOutputStream();
-    this.systemStream = new SystemStream(outputStream.getStreamSpec().getSystemName(),
-        outputStream.getStreamSpec().getPhysicalName());
+    this.systemStream = outputStream.getSystemStream();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/main/java/org/apache/samza/operators/impl/PartitionByOperatorImpl.java
----------------------------------------------------------------------
diff --git 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
index 9fc1e7c..dd64429 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
@@ -21,6 +21,7 @@ package org.apache.samza.operators.impl;
 import org.apache.samza.config.Config;
 import org.apache.samza.container.TaskContextImpl;
 import org.apache.samza.operators.KV;
+import org.apache.samza.operators.functions.MapFunction;
 import org.apache.samza.operators.spec.OperatorSpec;
 import org.apache.samza.operators.spec.OutputStreamImpl;
 import org.apache.samza.operators.spec.PartitionByOperatorSpec;
@@ -36,7 +37,6 @@ import org.apache.samza.task.TaskCoordinator;
 
 import java.util.Collection;
 import java.util.Collections;
-import java.util.function.Function;
 
 
 /**
@@ -46,17 +46,15 @@ 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 MapFunction<? super M, ? extends K> keyFunction;
+  private final MapFunction<? 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();
-    this.systemStream = new SystemStream(
-        outputStream.getStreamSpec().getSystemName(),
-        outputStream.getStreamSpec().getPhysicalName());
+    this.systemStream = outputStream.getSystemStream();
     this.keyFunction = partitionByOpSpec.getKeyFunction();
     this.valueFunction = partitionByOpSpec.getValueFunction();
     this.taskName = context.getTaskName().getTaskName();
@@ -66,6 +64,8 @@ class PartitionByOperatorImpl<M, K, V> extends OperatorImpl<M, Void> {
 
   @Override
   protected void handleInit(Config config, TaskContext context) {
+    this.keyFunction.init(config, context);
+    this.valueFunction.init(config, context);
   }
 
   @Override
@@ -80,6 +80,8 @@ class PartitionByOperatorImpl<M, K, V> extends OperatorImpl<M, Void> {
 
   @Override
   protected void handleClose() {
+    this.keyFunction.close();
+    this.valueFunction.close();
   }
 
   @Override
@@ -100,7 +102,7 @@ class PartitionByOperatorImpl<M, K, V> extends OperatorImpl<M, Void> {
   }
 
   private void sendControlMessage(ControlMessage message, MessageCollector collector) {
-    SystemStream outputStream = partitionByOpSpec.getOutputStream().getStreamSpec().toSystemStream();
+    SystemStream outputStream = partitionByOpSpec.getOutputStream().getSystemStream();
     controlMessageSender.send(message, outputStream, collector);
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java
index a51d5e6..6cd426b 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java
@@ -40,8 +40,7 @@ class StreamOperatorImpl<M, RM> extends OperatorImpl<M, RM> {
   private final StreamOperatorSpec<M, RM> streamOpSpec;
   private final FlatMapFunction<M, RM> transformFn;
 
-  StreamOperatorImpl(StreamOperatorSpec<M, RM> streamOpSpec,
-      Config config, TaskContext context) {
+  StreamOperatorImpl(StreamOperatorSpec<M, RM> streamOpSpec) {
     this.streamOpSpec = streamOpSpec;
     this.transformFn = streamOpSpec.getTransformFn();
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java
index 32406cb..6b5baae 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java
@@ -23,6 +23,8 @@ package org.apache.samza.operators.impl;
 import com.google.common.base.Preconditions;
 import org.apache.samza.config.Config;
 import org.apache.samza.operators.functions.FoldLeftFunction;
+import org.apache.samza.operators.functions.MapFunction;
+import org.apache.samza.operators.functions.SupplierFunction;
 import org.apache.samza.operators.impl.store.TimeSeriesKey;
 import org.apache.samza.operators.impl.store.TimeSeriesStore;
 import org.apache.samza.operators.impl.store.TimeSeriesStoreImpl;
@@ -58,8 +60,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
-import java.util.function.Function;
-import java.util.function.Supplier;
 import java.util.stream.Collectors;
 
 /**
@@ -93,8 +93,8 @@ public class WindowOperatorImpl<M, K> extends OperatorImpl<M, WindowPane<K, Obje
   private final Clock clock;
   private final WindowInternal<M, K, Object> window;
   private final FoldLeftFunction<M, Object> foldLeftFn;
-  private final Supplier<Object> initializer;
-  private final Function<M, K> keyFn;
+  private final SupplierFunction<Object> initializer;
+  private final MapFunction<M, K> keyFn;
 
   private final TriggerScheduler<K> triggerScheduler;
   private final Map<TriggerKey<K>, TriggerImplHandler> triggers = new HashMap<>();
@@ -112,11 +112,18 @@ public class WindowOperatorImpl<M, K> extends OperatorImpl<M, WindowPane<K, Obje
 
   @Override
   protected void handleInit(Config config, TaskContext context) {
-    WindowInternal<M, K, Object> window = windowOpSpec.getWindow();
 
     KeyValueStore<TimeSeriesKey<K>, Object> store =
         (KeyValueStore<TimeSeriesKey<K>, Object>) context.getStore(windowOpSpec.getOpId());
 
+    if (initializer != null) {
+      initializer.init(config, context);
+    }
+
+    if (keyFn != null) {
+      keyFn.init(config, context);
+    }
+
     // For aggregating windows, we use the store in over-write mode since we only retain the aggregated
     // value. Else, we use the store in append-mode.
     if (foldLeftFn != null) {
@@ -215,6 +222,12 @@ public class WindowOperatorImpl<M, K> extends OperatorImpl<M, WindowPane<K, Obje
     if (timeSeriesStore != null) {
       timeSeriesStore.close();
     }
+    if (initializer != null) {
+      initializer.close();
+    }
+    if (keyFn != null) {
+      keyFn.close();
+    }
   }
 
   private TriggerImplHandler getOrCreateTriggerImplHandler(TriggerKey<K> triggerKey, Trigger<M> trigger) {

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/main/java/org/apache/samza/operators/spec/FilterOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/FilterOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/FilterOperatorSpec.java
new file mode 100644
index 0000000..a5cdb82
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/FilterOperatorSpec.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 java.util.ArrayList;
+import java.util.Collection;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.functions.FilterFunction;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.functions.TimerFunction;
+import org.apache.samza.operators.functions.WatermarkFunction;
+import org.apache.samza.task.TaskContext;
+
+
+/**
+ * The spec for an operator that filters input messages based on some conditions.
+ *
+ * @param <M> type of input message
+ */
+class FilterOperatorSpec<M> extends StreamOperatorSpec<M, M> {
+  private final FilterFunction<M> filterFn;
+
+  FilterOperatorSpec(FilterFunction<M> filterFn, String opId) {
+    super(new FlatMapFunction<M, M>() {
+      @Override
+      public Collection<M> apply(M message) {
+        return new ArrayList<M>() {
+          {
+            if (filterFn.apply(message)) {
+              this.add(message);
+            }
+          }
+        };
+      }
+
+      @Override
+      public void init(Config config, TaskContext context) {
+        filterFn.init(config, context);
+      }
+
+      @Override
+      public void close() {
+        filterFn.close();
+      }
+    }, OpCode.FILTER, opId);
+    this.filterFn = filterFn;
+  }
+
+  @Override
+  public WatermarkFunction getWatermarkFn() {
+    return this.filterFn instanceof WatermarkFunction ? (WatermarkFunction) this.filterFn : null;
+  }
+
+  @Override
+  public TimerFunction getTimerFn() {
+    return this.filterFn instanceof TimerFunction ? (TimerFunction) this.filterFn : null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/main/java/org/apache/samza/operators/spec/FlatMapOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/FlatMapOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/FlatMapOperatorSpec.java
new file mode 100644
index 0000000..a93a221
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/FlatMapOperatorSpec.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.spec;
+
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.functions.TimerFunction;
+import org.apache.samza.operators.functions.WatermarkFunction;
+
+
+/**
+ * The spec for an operator that transforms each input message to a collection of output messages.
+ *
+ * @param <M> type of input message
+ * @param <OM> type of output messages
+ */
+class FlatMapOperatorSpec<M, OM> extends StreamOperatorSpec<M, OM> {
+
+  FlatMapOperatorSpec(FlatMapFunction<M, OM> flatMapFn, String opId) {
+    super(flatMapFn, OpCode.FLAT_MAP, opId);
+  }
+
+  @Override
+  public WatermarkFunction getWatermarkFn() {
+    return this.transformFn instanceof WatermarkFunction ? (WatermarkFunction) this.transformFn : null;
+  }
+
+  @Override
+  public TimerFunction getTimerFn() {
+    return this.transformFn instanceof TimerFunction ? (TimerFunction) this.transformFn : null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 2ed1e30..a636ac5 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
@@ -20,8 +20,8 @@ package org.apache.samza.operators.spec;
 
 import org.apache.samza.operators.KV;
 import org.apache.samza.operators.functions.TimerFunction;
-import org.apache.samza.serializers.Serde;
 import org.apache.samza.operators.functions.WatermarkFunction;
+import org.apache.samza.serializers.Serde;
 import org.apache.samza.system.StreamSpec;
 
 /**
@@ -33,10 +33,15 @@ import org.apache.samza.system.StreamSpec;
  */
 public class InputOperatorSpec<K, V> extends OperatorSpec<KV<K, V>, Object> { // Object == KV<K, V> | V
 
-  private final StreamSpec streamSpec;
-  private final Serde<K> keySerde;
-  private final Serde<V> valueSerde;
   private final boolean isKeyed;
+  private final StreamSpec streamSpec;
+
+  /**
+   * The following {@link Serde}s are serialized by the ExecutionPlanner when generating the configs for a stream, and deserialized
+   * once during startup in SamzaContainer. They don't need to be deserialized here on a per-task basis
+   */
+  private transient final Serde<K> keySerde;
+  private transient final Serde<V> valueSerde;
 
   public InputOperatorSpec(StreamSpec streamSpec,
       Serde<K> keySerde, Serde<V> valueSerde, boolean isKeyed, String opId) {

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 9e058ff..a218135 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
@@ -42,14 +42,20 @@ import java.util.Map;
  */
 public class JoinOperatorSpec<K, M, OM, JM> extends OperatorSpec<Object, JM> implements StatefulOperatorSpec { // Object == M | OM
 
-  private final OperatorSpec<?, M> leftInputOpSpec;
-  private final OperatorSpec<?, OM> rightInputOpSpec;
   private final JoinFunction<K, M, OM, JM> joinFn;
-  private final Serde<K> keySerde;
-  private final Serde<TimestampedValue<M>> messageSerde;
-  private final Serde<TimestampedValue<OM>> otherMessageSerde;
   private final long ttlMs;
 
+  private final OperatorSpec<?, M> leftInputOpSpec;
+  private final OperatorSpec<?, OM> rightInputOpSpec;
+
+  /**
+   * The following {@link Serde}s are serialized by the ExecutionPlanner when generating the store configs for a join, and
+   * deserialized once during startup in SamzaContainer. They don't need to be deserialized here on a per-task basis
+   */
+  private transient final Serde<K> keySerde;
+  private transient final Serde<TimestampedValue<M>> messageSerde;
+  private transient final Serde<TimestampedValue<OM>> otherMessageSerde;
+
   /**
    * Default constructor for a {@link JoinOperatorSpec}.
    *
@@ -126,4 +132,5 @@ public class JoinOperatorSpec<K, M, OM, JM> extends OperatorSpec<Object, JM> imp
   public long getTtlMs() {
     return ttlMs;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/main/java/org/apache/samza/operators/spec/MapOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/MapOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/MapOperatorSpec.java
new file mode 100644
index 0000000..1e2190b
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/MapOperatorSpec.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.spec;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.functions.MapFunction;
+import org.apache.samza.operators.functions.TimerFunction;
+import org.apache.samza.operators.functions.WatermarkFunction;
+import org.apache.samza.task.TaskContext;
+
+
+/**
+ * The spec for an operator that transforms each input message to a single output message.
+ *
+ * @param <M> type of input message
+ * @param <OM> type of output messages
+ */
+class MapOperatorSpec<M, OM> extends StreamOperatorSpec<M, OM> {
+
+  private final MapFunction<M, OM> mapFn;
+
+  MapOperatorSpec(MapFunction<M, OM> mapFn, String opId) {
+    super(new FlatMapFunction<M, OM>() {
+      @Override
+      public Collection<OM> apply(M message) {
+        return new ArrayList<OM>() {
+          {
+            OM r = mapFn.apply(message);
+            if (r != null) {
+              this.add(r);
+            }
+          }
+        };
+      }
+
+      @Override
+      public void init(Config config, TaskContext context) {
+        mapFn.init(config, context);
+      }
+
+      @Override
+      public void close() {
+        mapFn.close();
+      }
+    }, OpCode.MAP, opId);
+    this.mapFn = mapFn;
+  }
+
+  @Override
+  public WatermarkFunction getWatermarkFn() {
+    return this.mapFn instanceof WatermarkFunction ? (WatermarkFunction) this.mapFn : null;
+  }
+
+  @Override
+  public TimerFunction getTimerFn() {
+    return this.mapFn instanceof TimerFunction ? (TimerFunction) this.mapFn : null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/main/java/org/apache/samza/operators/spec/MergeOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/MergeOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/MergeOperatorSpec.java
new file mode 100644
index 0000000..987f72c
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/MergeOperatorSpec.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.spec;
+
+import java.util.ArrayList;
+import org.apache.samza.operators.functions.TimerFunction;
+import org.apache.samza.operators.functions.WatermarkFunction;
+
+
+/**
+ * The spec for an operator that combines messages from all input streams into a single output stream.
+ *
+ * @param <M> the type of messages in all input streams
+ */
+class MergeOperatorSpec<M> extends StreamOperatorSpec<M, M> {
+
+  MergeOperatorSpec(String opId) {
+    super((M message) ->
+        new ArrayList<M>() {
+        {
+          this.add(message);
+        }
+      }, OperatorSpec.OpCode.MERGE, opId);
+  }
+
+  @Override
+  public WatermarkFunction getWatermarkFn() {
+    return null;
+  }
+
+  @Override
+  public TimerFunction getTimerFn() {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 7b0a41b..e1e1c55 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
@@ -18,9 +18,9 @@
  */
 package org.apache.samza.operators.spec;
 
+import java.io.Serializable;
 import java.util.Collection;
 import java.util.LinkedHashSet;
-import java.util.Set;
 
 import org.apache.samza.annotation.InterfaceStability;
 import org.apache.samza.operators.MessageStream;
@@ -30,14 +30,14 @@ import org.apache.samza.operators.functions.WatermarkFunction;
 
 /**
  * A stream operator specification that holds all the information required to transform
- * the input {@link org.apache.samza.operators.MessageStreamImpl} and produce the output
- * {@link org.apache.samza.operators.MessageStreamImpl}.
+ * the input {@link MessageStreamImpl} and produce the output
+ * {@link MessageStreamImpl}.
  *
  * @param <M>  the type of input message to the operator
  * @param <OM>  the type of output message from the operator
  */
 @InterfaceStability.Unstable
-public abstract class OperatorSpec<M, OM> {
+public abstract class OperatorSpec<M, OM> implements Serializable {
 
   public enum OpCode {
     INPUT,
@@ -61,9 +61,15 @@ public abstract class OperatorSpec<M, OM> {
   /**
    * The set of operators that consume the messages produced from this operator.
    * <p>
-   * We use a LinkedHashSet since we need deterministic ordering in initializing/closing operators.
+   * We use a LinkedHashSet since we need both deterministic ordering in initializing/closing operators and serializability.
    */
-  private final Set<OperatorSpec<OM, ?>> nextOperatorSpecs = new LinkedHashSet<>();
+  private final LinkedHashSet<OperatorSpec<OM, ?>> nextOperatorSpecs = new LinkedHashSet<>();
+
+  // this method is used in unit tests to verify an {@link OperatorSpec} instance is a deserialized copy of this object.
+  final boolean isClone(OperatorSpec other) {
+    return this != other && this.getClass().isAssignableFrom(other.getClass())
+        && this.opCode.equals(other.opCode) && this.opId.equals(other.opId);
+  }
 
   public OperatorSpec(OpCode opCode, String opId) {
     this.opCode = opCode;
@@ -79,6 +85,11 @@ public abstract class OperatorSpec<M, OM> {
     nextOperatorSpecs.add(nextOperatorSpec);
   }
 
+  /**
+   * Get the collection of chained {@link OperatorSpec}s that are consuming the output of this node
+   *
+   * @return the collection of chained {@link OperatorSpec}s
+   */
   public Collection<OperatorSpec<OM, ?>> getRegisteredOperatorSpecs() {
     return nextOperatorSpecs;
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java
index c38f6e8..6e98d5a 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java
@@ -19,11 +19,6 @@
 
 package org.apache.samza.operators.spec;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.function.Function;
-
-import org.apache.samza.config.Config;
 import org.apache.samza.operators.KV;
 import org.apache.samza.operators.functions.FilterFunction;
 import org.apache.samza.operators.functions.FlatMapFunction;
@@ -35,7 +30,6 @@ import org.apache.samza.operators.windows.internal.WindowInternal;
 import org.apache.samza.serializers.Serde;
 import org.apache.samza.system.StreamSpec;
 import org.apache.samza.table.TableSpec;
-import org.apache.samza.task.TaskContext;
 
 
 /**
@@ -73,29 +67,7 @@ public class OperatorSpecs {
    */
   public static <M, OM> StreamOperatorSpec<M, OM> createMapOperatorSpec(
       MapFunction<? super M, ? extends OM> mapFn, String opId) {
-    return new StreamOperatorSpec<>(new FlatMapFunction<M, OM>() {
-      @Override
-      public Collection<OM> apply(M message) {
-        return new ArrayList<OM>() {
-          {
-            OM r = mapFn.apply(message);
-            if (r != null) {
-              this.add(r);
-            }
-          }
-        };
-      }
-
-      @Override
-      public void init(Config config, TaskContext context) {
-        mapFn.init(config, context);
-      }
-
-      @Override
-      public void close() {
-        mapFn.close();
-      }
-    }, mapFn, OperatorSpec.OpCode.MAP, opId);
+    return new MapOperatorSpec<>((MapFunction<M, OM>) mapFn, opId);
   }
 
   /**
@@ -108,28 +80,7 @@ public class OperatorSpecs {
    */
   public static <M> StreamOperatorSpec<M, M> createFilterOperatorSpec(
       FilterFunction<? super M> filterFn, String opId) {
-    return new StreamOperatorSpec<>(new FlatMapFunction<M, M>() {
-      @Override
-      public Collection<M> apply(M message) {
-        return new ArrayList<M>() {
-          {
-            if (filterFn.apply(message)) {
-              this.add(message);
-            }
-          }
-        };
-      }
-
-      @Override
-      public void init(Config config, TaskContext context) {
-        filterFn.init(config, context);
-      }
-
-      @Override
-      public void close() {
-        filterFn.close();
-      }
-    }, filterFn, OperatorSpec.OpCode.FILTER, opId);
+    return new FilterOperatorSpec<>((FilterFunction<M>) filterFn, opId);
   }
 
   /**
@@ -143,7 +94,7 @@ public class OperatorSpecs {
    */
   public static <M, OM> StreamOperatorSpec<M, OM> createFlatMapOperatorSpec(
       FlatMapFunction<? super M, ? extends OM> flatMapFn, String opId) {
-    return new StreamOperatorSpec<>((FlatMapFunction<M, OM>) flatMapFn, flatMapFn, OperatorSpec.OpCode.FLAT_MAP, opId);
+    return new FlatMapOperatorSpec<>((FlatMapFunction<M, OM>) flatMapFn, opId);
   }
 
   /**
@@ -183,8 +134,8 @@ public class OperatorSpecs {
    * @return  the {@link OutputOperatorSpec} for the partitionBy operator
    */
   public static <M, K, V> PartitionByOperatorSpec<M, K, V> createPartitionByOperatorSpec(
-      OutputStreamImpl<KV<K, V>> outputStream, Function<? super M, ? extends K> keyFunction,
-      Function<? super M, ? extends V> valueFunction, String opId) {
+      OutputStreamImpl<KV<K, V>> outputStream, MapFunction<? super M, ? extends K> keyFunction,
+      MapFunction<? super M, ? extends V> valueFunction, String opId) {
     return new PartitionByOperatorSpec<>(outputStream, keyFunction, valueFunction, opId);
   }
 
@@ -198,7 +149,6 @@ public class OperatorSpecs {
    * @param <WV>  the type of value in the window
    * @return  the {@link WindowOperatorSpec}
    */
-
   public static <M, WK, WV> WindowOperatorSpec<M, WK, WV> createWindowOperatorSpec(
       WindowInternal<M, WK, WV> window, String opId) {
     return new WindowOperatorSpec<>(window, opId);
@@ -236,13 +186,7 @@ public class OperatorSpecs {
    * @return  the {@link StreamOperatorSpec} for the merge
    */
   public static <M> StreamOperatorSpec<M, M> createMergeOperatorSpec(String opId) {
-    return new StreamOperatorSpec<>(message ->
-        new ArrayList<M>() {
-          {
-            this.add(message);
-          }
-        },
-        null, OperatorSpec.OpCode.MERGE, opId);
+    return new MergeOperatorSpec<>(opId);
   }
 
   /**
@@ -266,7 +210,6 @@ public class OperatorSpecs {
    * Creates a {@link SendToTableOperatorSpec} with a key extractor and a value extractor function,
    * the type of incoming message is expected to be KV&#60;K, V&#62;.
    *
-   * @param inputOpSpec the operator spec for the input stream
    * @param tableSpec the table spec for the underlying table
    * @param opId the unique ID of the operator
    * @param <K> the type of the table record key
@@ -274,8 +217,8 @@ public class OperatorSpecs {
    * @return the {@link SendToTableOperatorSpec}
    */
   public static <K, V> SendToTableOperatorSpec<K, V> createSendToTableOperatorSpec(
-      OperatorSpec<?, KV<K, V>> inputOpSpec, TableSpec tableSpec, String opId) {
-    return new SendToTableOperatorSpec(inputOpSpec, tableSpec, opId);
+     TableSpec tableSpec, String opId) {
+    return new SendToTableOperatorSpec(tableSpec, opId);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/main/java/org/apache/samza/operators/spec/OutputStreamImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/OutputStreamImpl.java b/samza-core/src/main/java/org/apache/samza/operators/spec/OutputStreamImpl.java
index e439c4e..fe0abcb 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/spec/OutputStreamImpl.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/OutputStreamImpl.java
@@ -18,18 +18,25 @@
  */
 package org.apache.samza.operators.spec;
 
+import java.io.Serializable;
 import org.apache.samza.operators.OutputStream;
 import org.apache.samza.serializers.Serde;
 import org.apache.samza.system.StreamSpec;
+import org.apache.samza.system.SystemStream;
 
 
-public class OutputStreamImpl<M> implements OutputStream<M> {
+public class OutputStreamImpl<M> implements OutputStream<M>, Serializable {
 
   private final StreamSpec streamSpec;
-  private final Serde keySerde;
-  private final Serde valueSerde;
   private final boolean isKeyed;
 
+  /**
+   * The following fields are serialized by the ExecutionPlanner when generating the configs for the output stream, and
+   * deserialized once during startup in SamzaContainer. They don't need to be deserialized here on a per-task basis
+   */
+  private transient final Serde keySerde;
+  private transient final Serde valueSerde;
+
   public OutputStreamImpl(StreamSpec streamSpec,
       Serde keySerde, Serde valueSerde, boolean isKeyed) {
     this.streamSpec = streamSpec;
@@ -50,6 +57,10 @@ public class OutputStreamImpl<M> implements OutputStream<M> {
     return valueSerde;
   }
 
+  public SystemStream getSystemStream() {
+    return this.streamSpec.toSystemStream();
+  }
+
   public boolean isKeyed() {
     return isKeyed;
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/main/java/org/apache/samza/operators/spec/PartitionByOperatorSpec.java
----------------------------------------------------------------------
diff --git 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
index a0a9b61..d6bf3d9 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
@@ -19,10 +19,11 @@
 package org.apache.samza.operators.spec;
 
 import org.apache.samza.operators.KV;
+import org.apache.samza.operators.functions.MapFunction;
 import org.apache.samza.operators.functions.TimerFunction;
 import org.apache.samza.operators.functions.WatermarkFunction;
 
-import java.util.function.Function;
+import static com.google.common.base.Preconditions.checkArgument;
 
 
 /**
@@ -39,21 +40,25 @@ import java.util.function.Function;
 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;
+  private final MapFunction<? super M, ? extends K> keyFunction;
+  private final MapFunction<? 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 keyFunction the {@link MapFunction} for extracting the key from the message
+   * @param valueFunction the {@link MapFunction} 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, String opId) {
+      MapFunction<? super M, ? extends K> keyFunction,
+      MapFunction<? super M, ? extends V> valueFunction, String opId) {
     super(OpCode.PARTITION_BY, opId);
+    checkArgument(!(keyFunction instanceof TimerFunction || keyFunction instanceof WatermarkFunction),
+        "keyFunction for partitionBy should not implement TimerFunction or WatermarkFunction.");
+    checkArgument(!(valueFunction instanceof TimerFunction || valueFunction instanceof WatermarkFunction),
+        "valueFunction for partitionBy should not implement TimerFunction or WatermarkFunction.");
     this.outputStream = outputStream;
     this.keyFunction = keyFunction;
     this.valueFunction = valueFunction;
@@ -67,11 +72,11 @@ public class PartitionByOperatorSpec<M, K, V> extends OperatorSpec<M, Void> {
     return this.outputStream;
   }
 
-  public Function<? super M, ? extends K> getKeyFunction() {
+  public MapFunction<? super M, ? extends K> getKeyFunction() {
     return keyFunction;
   }
 
-  public Function<? super M, ? extends V> getValueFunction() {
+  public MapFunction<? super M, ? extends V> getValueFunction() {
     return valueFunction;
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/main/java/org/apache/samza/operators/spec/SendToTableOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/SendToTableOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/SendToTableOperatorSpec.java
index e1b51be..22f393e 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/spec/SendToTableOperatorSpec.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/SendToTableOperatorSpec.java
@@ -35,26 +35,19 @@ import org.apache.samza.table.TableSpec;
 @InterfaceStability.Unstable
 public class SendToTableOperatorSpec<K, V> extends OperatorSpec<KV<K, V>, Void> {
 
-  private final OperatorSpec<?, KV<K, V>> inputOpSpec;
   private final TableSpec tableSpec;
 
   /**
    * Constructor for a {@link SendToTableOperatorSpec}.
    *
-   * @param inputOpSpec  the operator spec of the input stream
    * @param tableSpec  the table spec of the table written to
    * @param opId  the unique ID for this operator
    */
-  SendToTableOperatorSpec(OperatorSpec<?, KV<K, V>> inputOpSpec, TableSpec tableSpec, String opId) {
+  SendToTableOperatorSpec(TableSpec tableSpec, String opId) {
     super(OpCode.SEND_TO, opId);
-    this.inputOpSpec = inputOpSpec;
     this.tableSpec = tableSpec;
   }
 
-  public OperatorSpec<?, KV<K, V>> getInputOpSpec() {
-    return inputOpSpec;
-  }
-
   public TableSpec getTableSpec() {
     return tableSpec;
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 644eb6c..3addbf7 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,46 +19,31 @@
 package org.apache.samza.operators.spec;
 
 import org.apache.samza.operators.functions.FlatMapFunction;
-import org.apache.samza.operators.functions.TimerFunction;
-import org.apache.samza.operators.functions.WatermarkFunction;
-
 
 /**
- * The spec for a simple stream operator that outputs 0 or more messages for each input message.
+ * The common spec for a simple stream operator that outputs 0 or more messages for each input message.
  *
  * @param <M>  the type of input message
  * @param <OM>  the type of output message
  */
-public class StreamOperatorSpec<M, OM> extends OperatorSpec<M, OM> {
+public abstract class StreamOperatorSpec<M, OM> extends OperatorSpec<M, OM> {
 
-  private final FlatMapFunction<M, OM> transformFn;
-  private final Object originalFn;
+  protected final FlatMapFunction<M, OM> transformFn;
 
   /**
    * Constructor for a {@link StreamOperatorSpec}.
    *
    * @param transformFn  the transformation function
-   * @param originalFn the original user function before wrapping to transformFn
    * @param opCode  the {@link OpCode} for this {@link StreamOperatorSpec}
    * @param opId  the unique ID for this {@link StreamOperatorSpec}
    */
-  StreamOperatorSpec(FlatMapFunction<M, OM> transformFn, Object originalFn, OperatorSpec.OpCode opCode, String opId) {
+  protected StreamOperatorSpec(FlatMapFunction<M, OM> transformFn, OperatorSpec.OpCode opCode, String opId) {
     super(opCode, opId);
     this.transformFn = transformFn;
-    this.originalFn = originalFn;
   }
 
   public FlatMapFunction<M, OM> getTransformFn() {
     return this.transformFn;
   }
 
-  @Override
-  public WatermarkFunction getWatermarkFn() {
-    return originalFn instanceof WatermarkFunction ? (WatermarkFunction) originalFn : null;
-  }
-
-  @Override
-  public TimerFunction getTimerFn() {
-    return originalFn instanceof TimerFunction ? (TimerFunction) originalFn : null;
-  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 73d10ff..8d1ad29 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
@@ -40,6 +40,8 @@ import java.util.Collections;
 import java.util.List;
 import java.util.stream.Collectors;
 
+import static com.google.common.base.Preconditions.*;
+
 
 /**
  * The spec for an operator that groups messages into finite windows for processing
@@ -61,6 +63,15 @@ public class WindowOperatorSpec<M, WK, WV> extends OperatorSpec<M, WindowPane<WK
    */
   WindowOperatorSpec(WindowInternal<M, WK, WV> window, String opId) {
     super(OpCode.WINDOW, opId);
+    checkArgument(window.getInitializer() == null ||
+        !(window.getInitializer() instanceof TimerFunction || window.getInitializer() instanceof WatermarkFunction),
+        "A window does not accepts a user-defined TimerFunction or WatermarkFunction as the initializer.");
+    checkArgument(window.getKeyExtractor() == null ||
+        !(window.getKeyExtractor() instanceof TimerFunction || window.getKeyExtractor() instanceof WatermarkFunction),
+        "A window does not accepts a user-defined TimerFunction or WatermarkFunction as the keyExtractor.");
+    checkArgument(window.getEventTimeExtractor() == null ||
+        !(window.getEventTimeExtractor() instanceof TimerFunction || window.getEventTimeExtractor() instanceof WatermarkFunction),
+        "A window does not accepts a user-defined TimerFunction or WatermarkFunction as the eventTimeExtractor.");
     this.window = window;
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java b/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java
index 5eeca99..272ba63 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/stream/IntermediateMessageStreamImpl.java
@@ -20,7 +20,7 @@ package org.apache.samza.operators.stream;
 
 import org.apache.samza.operators.MessageStreamImpl;
 import org.apache.samza.operators.OutputStream;
-import org.apache.samza.operators.StreamGraphImpl;
+import org.apache.samza.operators.StreamGraphSpec;
 import org.apache.samza.operators.spec.InputOperatorSpec;
 import org.apache.samza.operators.spec.OperatorSpec;
 import org.apache.samza.operators.spec.OutputStreamImpl;
@@ -45,7 +45,7 @@ public class IntermediateMessageStreamImpl<M> extends MessageStreamImpl<M> imple
   private final OutputStreamImpl<M> outputStream;
   private final boolean isKeyed;
 
-  public IntermediateMessageStreamImpl(StreamGraphImpl graph, InputOperatorSpec<?, M> inputOperatorSpec,
+  public IntermediateMessageStreamImpl(StreamGraphSpec graph, InputOperatorSpec<?, M> inputOperatorSpec,
       OutputStreamImpl<M> outputStream) {
     super(graph, (OperatorSpec<?, M>) inputOperatorSpec);
     this.outputStream = outputStream;

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/main/java/org/apache/samza/operators/triggers/Cancellable.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/triggers/Cancellable.java b/samza-core/src/main/java/org/apache/samza/operators/triggers/Cancellable.java
index ca0ba67..96defd5 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/triggers/Cancellable.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/triggers/Cancellable.java
@@ -30,5 +30,5 @@ public interface Cancellable {
    *
    * @return the result of the cancelation
    */
-  public boolean cancel();
+  boolean cancel();
 }


[08/10] samza git commit: SAMZA-1659: Serializable OperatorSpec

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/main/java/org/apache/samza/operators/triggers/TriggerImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/triggers/TriggerImpl.java b/samza-core/src/main/java/org/apache/samza/operators/triggers/TriggerImpl.java
index 705cab7..b186cdb 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/triggers/TriggerImpl.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/triggers/TriggerImpl.java
@@ -46,14 +46,14 @@ public interface TriggerImpl<M, WK> {
    * @param message the incoming message
    * @param context the {@link TriggerScheduler} to schedule and cancel callbacks
    */
-  public void onMessage(M message, TriggerScheduler<WK> context);
+  void onMessage(M message, TriggerScheduler<WK> context);
 
   /**
    * Returns {@code true} if the current state of the trigger indicates that its condition
    * is satisfied and it is ready to fire.
    * @return if this trigger should fire.
    */
-  public boolean shouldFire();
+  boolean shouldFire();
 
   /**
    * Invoked when the execution of this {@link TriggerImpl} is canceled by an up-stream {@link TriggerImpl}.
@@ -61,6 +61,6 @@ public interface TriggerImpl<M, WK> {
    * No calls to {@link #onMessage(Object, TriggerScheduler)} or {@link #shouldFire()} will be invoked
    * after this invocation.
    */
-  public void cancel();
+  void cancel();
 
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 68962ce..5043977 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
@@ -29,7 +29,8 @@ import org.apache.samza.config.StreamConfig;
 import org.apache.samza.execution.ExecutionPlan;
 import org.apache.samza.execution.ExecutionPlanner;
 import org.apache.samza.execution.StreamManager;
-import org.apache.samza.operators.StreamGraphImpl;
+import org.apache.samza.operators.OperatorSpecGraph;
+import org.apache.samza.operators.StreamGraphSpec;
 import org.apache.samza.system.StreamSpec;
 import org.apache.samza.system.SystemAdmins;
 import org.slf4j.Logger;
@@ -44,7 +45,7 @@ import java.util.Set;
 
 
 /**
- * Defines common, core behavior for implementations of the {@link ApplicationRunner} API
+ * Defines common, core behavior for implementations of the {@link ApplicationRunner} API.
  */
 public abstract class AbstractApplicationRunner extends ApplicationRunner {
   private static final Logger log = LoggerFactory.getLogger(AbstractApplicationRunner.class);
@@ -52,8 +53,14 @@ public abstract class AbstractApplicationRunner extends ApplicationRunner {
   private final StreamManager streamManager;
   private final SystemAdmins systemAdmins;
 
+  /**
+   * The {@link ApplicationRunner} is supposed to run a single {@link StreamApplication} instance in the full life-cycle
+   */
+  protected final StreamGraphSpec graphSpec;
+
   public AbstractApplicationRunner(Config config) {
     super(config);
+    this.graphSpec = new StreamGraphSpec(this, config);
     this.systemAdmins = new SystemAdmins(config);
     this.streamManager = new StreamManager(systemAdmins);
   }
@@ -126,23 +133,23 @@ public abstract class AbstractApplicationRunner extends ApplicationRunner {
   /* package private */
   ExecutionPlan getExecutionPlan(StreamApplication app, String runId) throws Exception {
     // build stream graph
-    StreamGraphImpl streamGraph = new StreamGraphImpl(this, config);
-    app.init(streamGraph, config);
+    app.init(graphSpec, config);
 
+    OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph();
     // 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());
+    Set<StreamSpec> inputStreams = new HashSet<>(specGraph.getInputOperators().keySet());
+    inputStreams.removeAll(specGraph.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);
+    return planner.plan(specGraph);
   }
 
   /* package private for testing */

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 1284060..d64e57a 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
@@ -42,6 +42,7 @@ import org.apache.samza.coordinator.CoordinationUtils;
 import org.apache.samza.coordinator.DistributedLockWithState;
 import org.apache.samza.execution.ExecutionPlan;
 import org.apache.samza.job.ApplicationStatus;
+import org.apache.samza.operators.StreamGraphSpec;
 import org.apache.samza.processor.StreamProcessor;
 import org.apache.samza.processor.StreamProcessorLifecycleListener;
 import org.apache.samza.system.StreamSpec;
@@ -139,7 +140,7 @@ public class LocalApplicationRunner extends AbstractApplicationRunner {
     LOG.info("LocalApplicationRunner will run " + taskName);
     LocalStreamProcessorLifeCycleListener listener = new LocalStreamProcessorLifeCycleListener();
 
-    StreamProcessor processor = createStreamProcessor(jobConfig, null, listener);
+    StreamProcessor processor = createStreamProcessor(jobConfig, listener);
 
     numProcessorsToStart.set(1);
     listener.setProcessor(processor);
@@ -169,7 +170,7 @@ public class LocalApplicationRunner extends AbstractApplicationRunner {
       plan.getJobConfigs().forEach(jobConfig -> {
           LOG.debug("Starting job {} StreamProcessor with config {}", jobConfig.getName(), jobConfig);
           LocalStreamProcessorLifeCycleListener listener = new LocalStreamProcessorLifeCycleListener();
-          StreamProcessor processor = createStreamProcessor(jobConfig, app, listener);
+          StreamProcessor processor = createStreamProcessor(jobConfig, graphSpec, listener);
           listener.setProcessor(processor);
           processors.add(processor);
         });
@@ -284,15 +285,32 @@ public class LocalApplicationRunner extends AbstractApplicationRunner {
   /**
    * Create {@link StreamProcessor} based on {@link StreamApplication} and the config
    * @param config config
-   * @param app {@link StreamApplication}
    * @return {@link StreamProcessor]}
    */
   /* package private */
   StreamProcessor createStreamProcessor(
       Config config,
-      StreamApplication app,
       StreamProcessorLifecycleListener listener) {
-    Object taskFactory = TaskFactoryUtil.createTaskFactory(config, app, new LocalApplicationRunner(config));
+    Object taskFactory = TaskFactoryUtil.createTaskFactory(config);
+    return getStreamProcessorInstance(config, taskFactory, listener);
+  }
+
+  /**
+   * Create {@link StreamProcessor} based on {@link StreamApplication} and the config
+   * @param config config
+   * @param graphBuilder {@link StreamGraphSpec}
+   * @return {@link StreamProcessor]}
+   */
+  /* package private */
+  StreamProcessor createStreamProcessor(
+      Config config,
+      StreamGraphSpec graphBuilder,
+      StreamProcessorLifecycleListener listener) {
+    Object taskFactory = TaskFactoryUtil.createTaskFactory(graphBuilder.getOperatorSpecGraph(), graphBuilder.getContextManager());
+    return getStreamProcessorInstance(config, taskFactory, listener);
+  }
+
+  private StreamProcessor getStreamProcessorInstance(Config config, Object taskFactory, StreamProcessorLifecycleListener listener) {
     if (taskFactory instanceof StreamTaskFactory) {
       return new StreamProcessor(
           config, new HashMap<>(), (StreamTaskFactory) taskFactory, listener);

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 5831910..7751241 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
@@ -70,8 +70,7 @@ public class LocalContainerRunner extends AbstractApplicationRunner {
 
   @Override
   public void run(StreamApplication streamApp) {
-    super.run(streamApp);
-    Object taskFactory = TaskFactoryUtil.createTaskFactory(config, streamApp, this);
+    Object taskFactory = getTaskFactory(streamApp);
 
     container = SamzaContainer$.MODULE$.apply(
         containerId,
@@ -106,6 +105,14 @@ public class LocalContainerRunner extends AbstractApplicationRunner {
     }
   }
 
+  private Object getTaskFactory(StreamApplication streamApp) {
+    if (streamApp != null) {
+      streamApp.init(graphSpec, config);
+      return TaskFactoryUtil.createTaskFactory(graphSpec.getOperatorSpecGraph(), graphSpec.getContextManager());
+    }
+    return TaskFactoryUtil.createTaskFactory(config);
+  }
+
   @Override
   public void kill(StreamApplication streamApp) {
     // Ultimately this class probably won't end up extending ApplicationRunner, so this will be deleted

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 e4b3c62..fdd134f 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
@@ -18,16 +18,14 @@
  */
 package org.apache.samza.task;
 
-import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.Config;
+import org.apache.samza.operators.OperatorSpecGraph;
 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;
-import org.apache.samza.runtime.ApplicationRunner;
 import org.apache.samza.system.IncomingMessageEnvelope;
 import org.apache.samza.system.SystemStream;
 import org.apache.samza.system.WatermarkMessage;
@@ -39,41 +37,45 @@ 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}.
+ * feeds the input messages into the user-defined transformation chains in {@link OperatorSpecGraph}.
  */
 public class StreamOperatorTask implements StreamTask, InitableTask, WindowableTask, ClosableTask {
   private static final Logger LOG = LoggerFactory.getLogger(StreamOperatorTask.class);
 
-  private final StreamApplication streamApplication;
-  private final ApplicationRunner runner;
+  private final OperatorSpecGraph specGraph;
+  // TODO: to be replaced by proper scope of shared context factory in SAMZA-1714
+  private final ContextManager contextManager;
   private final Clock clock;
 
   private OperatorImplGraph operatorImplGraph;
-  private ContextManager contextManager;
 
   /**
-   * Constructs an adaptor task to run the user-implemented {@link StreamApplication}.
-   * @param streamApplication the user-implemented {@link StreamApplication} that creates the logical DAG
-   * @param runner the {@link ApplicationRunner} to get the mapping between logical and physical streams
+   * Constructs an adaptor task to run the user-implemented {@link OperatorSpecGraph}.
+   * @param specGraph the serialized version of user-implemented {@link OperatorSpecGraph}
+   *                  that includes the logical DAG
+   * @param contextManager the {@link ContextManager} used to set up the shared context used by operators in the DAG
    * @param clock the {@link Clock} to use for time-keeping
    */
-  public StreamOperatorTask(StreamApplication streamApplication, ApplicationRunner runner, Clock clock) {
-    this.streamApplication = streamApplication;
-    this.runner = runner;
+  public StreamOperatorTask(OperatorSpecGraph specGraph, ContextManager contextManager, Clock clock) {
+    this.specGraph = specGraph.clone();
+    this.contextManager = contextManager;
     this.clock = clock;
   }
 
-  public StreamOperatorTask(StreamApplication application, ApplicationRunner runner) {
-    this(application, runner, SystemClock.instance());
+  public StreamOperatorTask(OperatorSpecGraph specGraph, ContextManager contextManager) {
+    this(specGraph, contextManager, SystemClock.instance());
   }
 
   /**
    * Initializes this task during startup.
    * <p>
-   * Implementation: Initializes the user-implemented {@link StreamApplication}. The {@link StreamApplication} sets
-   * the input and output streams and the task-wide context manager using the {@link StreamGraphImpl} APIs,
-   * and the logical transforms using the {@link org.apache.samza.operators.MessageStream} APIs. It then uses
-   * the {@link StreamGraphImpl} to create the {@link OperatorImplGraph} corresponding to the logical DAG.
+   * Implementation: Initializes the runtime {@link OperatorImplGraph} according to user-defined {@link OperatorSpecGraph}.
+   * The {@link org.apache.samza.operators.StreamGraphSpec} sets the input and output streams and the task-wide
+   * context manager using the {@link org.apache.samza.operators.StreamGraph} APIs,
+   * and the logical transforms using the {@link org.apache.samza.operators.MessageStream} APIs. After the
+   * {@link org.apache.samza.operators.StreamGraphSpec} is initialized once by the application, it then creates
+   * an immutable {@link OperatorSpecGraph} accordingly, which is passed in to this class to create the {@link OperatorImplGraph}
+   * corresponding to the logical DAG.
    *
    * @param config allows accessing of fields in the configuration files that this StreamTask is specified in
    * @param context allows initializing and accessing contextual data of this StreamTask
@@ -81,18 +83,14 @@ public class StreamOperatorTask implements StreamTask, InitableTask, WindowableT
    */
   @Override
   public final void init(Config config, TaskContext context) throws Exception {
-    StreamGraphImpl streamGraph = new StreamGraphImpl(runner, config);
-    // initialize the user-implemented stream application.
-    this.streamApplication.init(streamGraph, config);
 
-    // get the user-implemented context manager and initialize it
-    this.contextManager = streamGraph.getContextManager();
+    // get the user-implemented per task context manager and initialize it
     if (this.contextManager != null) {
       this.contextManager.init(config, context);
     }
 
     // create the operator impl DAG corresponding to the logical operator spec DAG
-    this.operatorImplGraph = new OperatorImplGraph(streamGraph, config, context, clock);
+    this.operatorImplGraph = new OperatorImplGraph(specGraph, config, context, clock);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java
index 2a894ae..38ae854 100644
--- a/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java
+++ b/samza-core/src/main/java/org/apache/samza/task/TaskFactoryUtil.java
@@ -24,7 +24,8 @@ import org.apache.samza.config.Config;
 import org.apache.samza.config.ConfigException;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.TaskConfig;
-import org.apache.samza.runtime.ApplicationRunner;
+import org.apache.samza.operators.ContextManager;
+import org.apache.samza.operators.OperatorSpecGraph;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -41,19 +42,28 @@ public class TaskFactoryUtil {
   private static final Logger log = LoggerFactory.getLogger(TaskFactoryUtil.class);
 
   /**
-   * This method creates a task factory class based on the configuration and {@link StreamApplication}
+   * This method creates a task factory class based on the {@link StreamApplication}
+   *
+   * @param specGraph the {@link OperatorSpecGraph}
+   * @param contextManager the {@link ContextManager} to set up initial context for {@code specGraph}
+   * @return  a task factory object, either a instance of {@link StreamTaskFactory} or {@link AsyncStreamTaskFactory}
+   */
+  public static Object createTaskFactory(OperatorSpecGraph specGraph, ContextManager contextManager) {
+    return createStreamOperatorTaskFactory(specGraph, contextManager);
+  }
+
+  /**
+   * This method creates a task factory class based on the configuration
    *
    * @param config  the {@link Config} for this job
-   * @param streamApp the {@link StreamApplication}
-   * @param runner  the {@link ApplicationRunner} to run this job
    * @return  a task factory object, either a instance of {@link StreamTaskFactory} or {@link AsyncStreamTaskFactory}
    */
-  public static Object createTaskFactory(Config config, StreamApplication streamApp, ApplicationRunner runner) {
-    return (streamApp != null) ? createStreamOperatorTaskFactory(streamApp, runner) : fromTaskClassConfig(config);
+  public static Object createTaskFactory(Config config) {
+    return fromTaskClassConfig(config);
   }
 
-  private static StreamTaskFactory createStreamOperatorTaskFactory(StreamApplication streamApp, ApplicationRunner runner) {
-    return () -> new StreamOperatorTask(streamApp, runner);
+  private static StreamTaskFactory createStreamOperatorTaskFactory(OperatorSpecGraph specGraph, ContextManager contextManager) {
+    return () -> new StreamOperatorTask(specGraph, contextManager);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 61e8c77..64ee7f3 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
@@ -28,7 +28,6 @@ import org.apache.samza.config.Config
 import org.apache.samza.config.StreamConfig.Config2Stream
 import org.apache.samza.job.model.JobModel
 import org.apache.samza.metrics.MetricsReporter
-import org.apache.samza.operators.functions.TimerFunction
 import org.apache.samza.storage.TaskStorageManager
 import org.apache.samza.system._
 import org.apache.samza.table.TableManager

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 e5ce3c8..029b375 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
@@ -27,10 +27,12 @@ import org.apache.samza.coordinator.JobModelManager
 import org.apache.samza.coordinator.stream.CoordinatorStreamManager
 import org.apache.samza.job.{StreamJob, StreamJobFactory}
 import org.apache.samza.metrics.{JmxServer, MetricsRegistryMap, MetricsReporter}
+import org.apache.samza.operators.StreamGraphSpec
 import org.apache.samza.runtime.LocalContainerRunner
 import org.apache.samza.storage.ChangelogStreamManager
 import org.apache.samza.task.TaskFactoryUtil
 import org.apache.samza.util.Logging
+
 import scala.collection.JavaConversions._
 import scala.collection.mutable
 
@@ -71,7 +73,14 @@ class ThreadJobFactory extends StreamJobFactory with Logging {
     val jmxServer = new JmxServer
     val streamApp = TaskFactoryUtil.createStreamApplication(config)
     val appRunner = new LocalContainerRunner(jobModel, "0")
-    val taskFactory = TaskFactoryUtil.createTaskFactory(config, streamApp, appRunner)
+
+    val taskFactory = if (streamApp != null) {
+      val graphSpec = new StreamGraphSpec(appRunner, config)
+      streamApp.init(graphSpec, config)
+      TaskFactoryUtil.createTaskFactory(graphSpec.getOperatorSpecGraph(), graphSpec.getContextManager)
+    } else {
+      TaskFactoryUtil.createTaskFactory(config)
+    }
 
     // Give developers a nice friendly warning if they've specified task.opts and are using a threaded job.
     config.getTaskOpts match {

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/example/BroadcastExample.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/example/BroadcastExample.java b/samza-core/src/test/java/org/apache/samza/example/BroadcastExample.java
deleted file mode 100644
index 7061732..0000000
--- a/samza-core/src/test/java/org/apache/samza/example/BroadcastExample.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.example;
-
-import org.apache.samza.application.StreamApplication;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.KV;
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.OutputStream;
-import org.apache.samza.operators.StreamGraph;
-import org.apache.samza.runtime.LocalApplicationRunner;
-import org.apache.samza.serializers.JsonSerdeV2;
-import org.apache.samza.serializers.KVSerde;
-import org.apache.samza.serializers.StringSerde;
-import org.apache.samza.util.CommandLine;
-
-
-/**
- * Example implementation of a task that splits its input into multiple output streams.
- */
-public class BroadcastExample implements StreamApplication {
-
-  @Override
-  public void init(StreamGraph graph, Config config) {
-    graph.setDefaultSerde(KVSerde.of(new StringSerde(), new JsonSerdeV2<>(PageViewEvent.class)));
-
-    MessageStream<KV<String, PageViewEvent>> inputStream = graph.getInputStream("inputStream");
-    OutputStream<KV<String, PageViewEvent>> outputStream1 = graph.getOutputStream("outputStream1");
-    OutputStream<KV<String, PageViewEvent>> outputStream2 = graph.getOutputStream("outputStream2");
-    OutputStream<KV<String, PageViewEvent>> outputStream3 = graph.getOutputStream("outputStream3");
-
-    inputStream.filter(m -> m.key.equals("key1")).sendTo(outputStream1);
-    inputStream.filter(m -> m.key.equals("key2")).sendTo(outputStream2);
-    inputStream.filter(m -> m.key.equals("key3")).sendTo(outputStream3);
-  }
-
-  // local execution mode
-  public static void main(String[] args) throws Exception {
-    CommandLine cmdLine = new CommandLine();
-    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
-    LocalApplicationRunner localRunner = new LocalApplicationRunner(config);
-    localRunner.run(new BroadcastExample());
-  }
-
-  class PageViewEvent {
-    String key;
-    long timestamp;
-
-    public PageViewEvent(String key, long timestamp) {
-      this.key = key;
-      this.timestamp = timestamp;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/example/KeyValueStoreExample.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/example/KeyValueStoreExample.java b/samza-core/src/test/java/org/apache/samza/example/KeyValueStoreExample.java
deleted file mode 100644
index f9e0a3a..0000000
--- a/samza-core/src/test/java/org/apache/samza/example/KeyValueStoreExample.java
+++ /dev/null
@@ -1,131 +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.example;
-
-
-import org.apache.samza.application.StreamApplication;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.KV;
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.OutputStream;
-import org.apache.samza.operators.StreamGraph;
-import org.apache.samza.operators.functions.FlatMapFunction;
-import org.apache.samza.runtime.LocalApplicationRunner;
-import org.apache.samza.serializers.JsonSerdeV2;
-import org.apache.samza.serializers.KVSerde;
-import org.apache.samza.serializers.StringSerde;
-import org.apache.samza.storage.kv.KeyValueStore;
-import org.apache.samza.task.TaskContext;
-import org.apache.samza.util.CommandLine;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-
-/**
- * Example code using {@link KeyValueStore} to implement event-time window
- */
-public class KeyValueStoreExample implements StreamApplication {
-
-  @Override public void init(StreamGraph graph, Config config) {
-    MessageStream<PageViewEvent> pageViewEvents =
-        graph.getInputStream("pageViewEventStream", new JsonSerdeV2<>(PageViewEvent.class));
-    OutputStream<KV<String, StatsOutput>> pageViewEventPerMember =
-        graph.getOutputStream("pageViewEventPerMember",
-            KVSerde.of(new StringSerde(), new JsonSerdeV2<>(StatsOutput.class)));
-
-    pageViewEvents
-        .partitionBy(pve -> pve.memberId, pve -> pve,
-            KVSerde.of(new StringSerde(), new JsonSerdeV2<>(PageViewEvent.class)), "partitionBy")
-        .map(KV::getValue)
-        .flatMap(new MyStatsCounter())
-        .map(stats -> KV.of(stats.memberId, stats))
-        .sendTo(pageViewEventPerMember);
-  }
-
-  // local execution mode
-  public static void main(String[] args) throws Exception {
-    CommandLine cmdLine = new CommandLine();
-    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
-    LocalApplicationRunner localRunner = new LocalApplicationRunner(config);
-    localRunner.run(new KeyValueStoreExample());
-  }
-
-  class MyStatsCounter implements FlatMapFunction<PageViewEvent, StatsOutput> {
-    private final int timeoutMs = 10 * 60 * 1000;
-
-    KeyValueStore<String, StatsWindowState> statsStore;
-
-    class StatsWindowState {
-      int lastCount = 0;
-      long timeAtLastOutput = 0;
-      int newCount = 0;
-    }
-
-    @Override
-    public Collection<StatsOutput> apply(PageViewEvent message) {
-      List<StatsOutput> outputStats = new ArrayList<>();
-      long wndTimestamp = (long) Math.floor(TimeUnit.MILLISECONDS.toMinutes(message.timestamp) / 5) * 5;
-      String wndKey = String.format("%s-%d", message.memberId, wndTimestamp);
-      StatsWindowState curState = this.statsStore.get(wndKey);
-      curState.newCount++;
-      long curTimeMs = System.currentTimeMillis();
-      if (curState.newCount > 0 && curState.timeAtLastOutput + timeoutMs < curTimeMs) {
-        curState.timeAtLastOutput = curTimeMs;
-        curState.lastCount += curState.newCount;
-        curState.newCount = 0;
-        outputStats.add(new StatsOutput(message.memberId, wndTimestamp, curState.lastCount));
-      }
-      // update counter w/o generating output
-      this.statsStore.put(wndKey, curState);
-      return outputStats;
-    }
-
-    @Override
-    public void init(Config config, TaskContext context) {
-      this.statsStore = (KeyValueStore<String, StatsWindowState>) context.getStore("my-stats-wnd-store");
-    }
-  }
-
-  class PageViewEvent {
-    String pageId;
-    String memberId;
-    long timestamp;
-
-    PageViewEvent(String pageId, String memberId, long timestamp) {
-      this.pageId = pageId;
-      this.memberId = memberId;
-      this.timestamp = timestamp;
-    }
-  }
-
-  class StatsOutput {
-    private String memberId;
-    private long timestamp;
-    private Integer count;
-
-    StatsOutput(String key, long timestamp, Integer count) {
-      this.memberId = key;
-      this.timestamp = timestamp;
-      this.count = count;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/example/MergeExample.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/example/MergeExample.java b/samza-core/src/test/java/org/apache/samza/example/MergeExample.java
deleted file mode 100644
index 4702c9a..0000000
--- a/samza-core/src/test/java/org/apache/samza/example/MergeExample.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.example;
-
-import com.google.common.collect.ImmutableList;
-import org.apache.samza.application.StreamApplication;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.KV;
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.OutputStream;
-import org.apache.samza.operators.StreamGraph;
-import org.apache.samza.runtime.LocalApplicationRunner;
-import org.apache.samza.serializers.IntegerSerde;
-import org.apache.samza.serializers.KVSerde;
-import org.apache.samza.serializers.StringSerde;
-import org.apache.samza.util.CommandLine;
-
-public class MergeExample implements StreamApplication {
-
-  @Override
-  public void init(StreamGraph graph, Config config) {
-    graph.setDefaultSerde(new StringSerde());
-
-    MessageStream<String> inputStream1 = graph.getInputStream("inputStream1");
-    MessageStream<String> inputStream2 = graph.getInputStream("inputStream2");
-    MessageStream<String> inputStream3 = graph.getInputStream("inputStream3");
-    OutputStream<KV<Integer, String>> outputStream =
-        graph.getOutputStream("outputStream", KVSerde.of(new IntegerSerde(), new StringSerde()));
-
-    MessageStream
-        .mergeAll(ImmutableList.of(inputStream1, inputStream2, inputStream3))
-        .map(m -> KV.of(m.hashCode(), m))
-        .sendTo(outputStream);
-  }
-
-  // local execution mode
-  public static void main(String[] args) throws Exception {
-    CommandLine cmdLine = new CommandLine();
-    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
-    LocalApplicationRunner localRunner = new LocalApplicationRunner(config);
-    localRunner.run(new MergeExample());
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java b/samza-core/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java
deleted file mode 100644
index ff785d9..0000000
--- a/samza-core/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.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.example;
-
-import org.apache.samza.application.StreamApplication;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.KV;
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.OutputStream;
-import org.apache.samza.operators.StreamGraph;
-import org.apache.samza.operators.functions.JoinFunction;
-import org.apache.samza.runtime.LocalApplicationRunner;
-import org.apache.samza.serializers.JsonSerdeV2;
-import org.apache.samza.serializers.KVSerde;
-import org.apache.samza.serializers.StringSerde;
-import org.apache.samza.util.CommandLine;
-
-import java.time.Duration;
-
-/**
- * Simple 2-way stream-to-stream join example
- */
-public class OrderShipmentJoinExample implements StreamApplication {
-
-  @Override
-  public void init(StreamGraph graph, Config config) {
-    MessageStream<OrderRecord> orders =
-        graph.getInputStream("orders", new JsonSerdeV2<>(OrderRecord.class));
-    MessageStream<ShipmentRecord> shipments =
-        graph.getInputStream("shipments", new JsonSerdeV2<>(ShipmentRecord.class));
-    OutputStream<KV<String, FulfilledOrderRecord>> fulfilledOrders =
-        graph.getOutputStream("fulfilledOrders",
-            KVSerde.of(new StringSerde(), new JsonSerdeV2<>(FulfilledOrderRecord.class)));
-
-    orders
-        .join(shipments, new MyJoinFunction(),
-            new StringSerde(), new JsonSerdeV2<>(OrderRecord.class), new JsonSerdeV2<>(ShipmentRecord.class),
-            Duration.ofMinutes(1), "join")
-        .map(fulFilledOrder -> KV.of(fulFilledOrder.orderId, fulFilledOrder))
-        .sendTo(fulfilledOrders);
-  }
-
-  // local execution mode
-  public static void main(String[] args) throws Exception {
-    CommandLine cmdLine = new CommandLine();
-    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
-    LocalApplicationRunner localRunner = new LocalApplicationRunner(config);
-    localRunner.run(new OrderShipmentJoinExample());
-  }
-
-  class MyJoinFunction implements JoinFunction<String, OrderRecord, ShipmentRecord, FulfilledOrderRecord> {
-    @Override
-    public FulfilledOrderRecord apply(OrderRecord message, ShipmentRecord otherMessage) {
-      return new FulfilledOrderRecord(message.orderId, message.orderTimeMs, otherMessage.shipTimeMs);
-    }
-
-    @Override
-    public String getFirstKey(OrderRecord message) {
-      return message.orderId;
-    }
-
-    @Override
-    public String getSecondKey(ShipmentRecord message) {
-      return message.orderId;
-    }
-  }
-
-  class OrderRecord {
-    String orderId;
-    long orderTimeMs;
-
-    OrderRecord(String orderId, long timeMs) {
-      this.orderId = orderId;
-      this.orderTimeMs = timeMs;
-    }
-  }
-
-  class ShipmentRecord {
-    String orderId;
-    long shipTimeMs;
-
-    ShipmentRecord(String orderId, long timeMs) {
-      this.orderId = orderId;
-      this.shipTimeMs = timeMs;
-    }
-  }
-
-  class FulfilledOrderRecord {
-    String orderId;
-    long orderTimeMs;
-    long shipTimeMs;
-
-    FulfilledOrderRecord(String orderId, long orderTimeMs, long shipTimeMs) {
-      this.orderId = orderId;
-      this.orderTimeMs = orderTimeMs;
-      this.shipTimeMs = shipTimeMs;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/example/PageViewCounterExample.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/example/PageViewCounterExample.java b/samza-core/src/test/java/org/apache/samza/example/PageViewCounterExample.java
deleted file mode 100644
index 846b9f8..0000000
--- a/samza-core/src/test/java/org/apache/samza/example/PageViewCounterExample.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.example;
-
-import org.apache.samza.application.StreamApplication;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.KV;
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.OutputStream;
-import org.apache.samza.operators.StreamGraph;
-import org.apache.samza.operators.functions.FoldLeftFunction;
-import org.apache.samza.operators.triggers.Triggers;
-import org.apache.samza.operators.windows.AccumulationMode;
-import org.apache.samza.operators.windows.WindowPane;
-import org.apache.samza.operators.windows.Windows;
-import org.apache.samza.runtime.LocalApplicationRunner;
-import org.apache.samza.serializers.JsonSerdeV2;
-import org.apache.samza.serializers.KVSerde;
-import org.apache.samza.serializers.StringSerde;
-import org.apache.samza.util.CommandLine;
-
-import java.time.Duration;
-import java.util.function.Supplier;
-
-
-/**
- * Example code to implement window-based counter
- */
-public class PageViewCounterExample implements StreamApplication {
-
-  @Override public void init(StreamGraph graph, Config config) {
-    MessageStream<PageViewEvent> pageViewEvents =
-        graph.getInputStream("pageViewEventStream", new JsonSerdeV2<>(PageViewEvent.class));
-    OutputStream<KV<String, PageViewCount>> pageViewEventPerMemberStream =
-        graph.getOutputStream("pageViewEventPerMemberStream",
-            KVSerde.of(new StringSerde(), new JsonSerdeV2<>(PageViewCount.class)));
-
-    Supplier<Integer> initialValue = () -> 0;
-    FoldLeftFunction<PageViewEvent, Integer> foldLeftFn = (m, c) -> c + 1;
-    pageViewEvents
-        .window(Windows.keyedTumblingWindow(m -> m.memberId, Duration.ofSeconds(10), initialValue, foldLeftFn, null, null)
-            .setEarlyTrigger(Triggers.repeat(Triggers.count(5)))
-            .setAccumulationMode(AccumulationMode.DISCARDING), "tumblingWindow")
-        .map(windowPane -> KV.of(windowPane.getKey().getKey(), new PageViewCount(windowPane)))
-        .sendTo(pageViewEventPerMemberStream);
-  }
-
-  // local execution mode
-  public static void main(String[] args) {
-    CommandLine cmdLine = new CommandLine();
-    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
-    LocalApplicationRunner localRunner = new LocalApplicationRunner(config);
-    localRunner.run(new PageViewCounterExample());
-  }
-
-  class PageViewEvent {
-    String pageId;
-    String memberId;
-    long timestamp;
-
-    PageViewEvent(String pageId, String memberId, long timestamp) {
-      this.pageId = pageId;
-      this.memberId = memberId;
-      this.timestamp = timestamp;
-    }
-  }
-
-  class PageViewCount {
-    String memberId;
-    long timestamp;
-    int count;
-
-    PageViewCount(WindowPane<String, Integer> m) {
-      this.memberId = m.getKey().getKey();
-      this.timestamp = Long.valueOf(m.getKey().getPaneId());
-      this.count = m.getMessage();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/example/RepartitionExample.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/example/RepartitionExample.java b/samza-core/src/test/java/org/apache/samza/example/RepartitionExample.java
deleted file mode 100644
index c9bcc45..0000000
--- a/samza-core/src/test/java/org/apache/samza/example/RepartitionExample.java
+++ /dev/null
@@ -1,90 +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.example;
-
-import org.apache.samza.application.StreamApplication;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.KV;
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.OutputStream;
-import org.apache.samza.operators.StreamGraph;
-import org.apache.samza.operators.windows.WindowPane;
-import org.apache.samza.operators.windows.Windows;
-import org.apache.samza.runtime.LocalApplicationRunner;
-import org.apache.samza.serializers.JsonSerdeV2;
-import org.apache.samza.serializers.KVSerde;
-import org.apache.samza.serializers.StringSerde;
-import org.apache.samza.util.CommandLine;
-
-import java.time.Duration;
-
-
-/**
- * Example {@link StreamApplication} code to test the API methods with re-partition operator
- */
-public class RepartitionExample implements StreamApplication {
-
-  @Override public void init(StreamGraph graph, Config config) {
-    MessageStream<PageViewEvent> pageViewEvents =
-        graph.getInputStream("pageViewEvent", new JsonSerdeV2<>(PageViewEvent.class));
-    OutputStream<KV<String, MyStreamOutput>> pageViewEventPerMember =
-        graph.getOutputStream("pageViewEventPerMember",
-            KVSerde.of(new StringSerde(), new JsonSerdeV2<>(MyStreamOutput.class)));
-
-    pageViewEvents
-        .partitionBy(pve -> pve.memberId, pve -> pve,
-            KVSerde.of(new StringSerde(), new JsonSerdeV2<>(PageViewEvent.class)), "partitionBy")
-        .window(Windows.keyedTumblingWindow(KV::getKey, Duration.ofMinutes(5), () -> 0, (m, c) -> c + 1, null, null),
-            "window")
-        .map(windowPane -> KV.of(windowPane.getKey().getKey(), new MyStreamOutput(windowPane)))
-        .sendTo(pageViewEventPerMember);
-  }
-
-  // local execution mode
-  public static void main(String[] args) throws Exception {
-    CommandLine cmdLine = new CommandLine();
-    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
-    LocalApplicationRunner localRunner = new LocalApplicationRunner(config);
-    localRunner.run(new RepartitionExample());
-  }
-
-  class PageViewEvent {
-    String pageId;
-    String memberId;
-    long timestamp;
-
-    PageViewEvent(String pageId, String memberId, long timestamp) {
-      this.pageId = pageId;
-      this.memberId = memberId;
-      this.timestamp = timestamp;
-    }
-  }
-
-  class MyStreamOutput {
-    String memberId;
-    long timestamp;
-    int count;
-
-    MyStreamOutput(WindowPane<String, Integer> m) {
-      this.memberId = m.getKey().getKey();
-      this.timestamp = Long.valueOf(m.getKey().getPaneId());
-      this.count = m.getMessage();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/example/WindowExample.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/example/WindowExample.java b/samza-core/src/test/java/org/apache/samza/example/WindowExample.java
deleted file mode 100644
index 3c37c31..0000000
--- a/samza-core/src/test/java/org/apache/samza/example/WindowExample.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.example;
-
-import org.apache.samza.application.StreamApplication;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.OutputStream;
-import org.apache.samza.operators.StreamGraph;
-import org.apache.samza.operators.functions.FoldLeftFunction;
-import org.apache.samza.operators.triggers.Triggers;
-import org.apache.samza.operators.windows.WindowPane;
-import org.apache.samza.operators.windows.Windows;
-import org.apache.samza.runtime.LocalApplicationRunner;
-import org.apache.samza.serializers.IntegerSerde;
-import org.apache.samza.serializers.JsonSerdeV2;
-import org.apache.samza.util.CommandLine;
-
-import java.time.Duration;
-import java.util.function.Supplier;
-
-
-/**
- * Example implementation of a simple user-defined task w/ a window operator.
- *
- */
-public class WindowExample implements StreamApplication {
-
-  @Override
-  public void init(StreamGraph graph, Config config) {
-    Supplier<Integer> initialValue = () -> 0;
-    FoldLeftFunction<PageViewEvent, Integer> counter = (m, c) -> c == null ? 1 : c + 1;
-    MessageStream<PageViewEvent> inputStream = graph.getInputStream("inputStream", new JsonSerdeV2<PageViewEvent>());
-    OutputStream<Integer> outputStream = graph.getOutputStream("outputStream", new IntegerSerde());
-
-    // create a tumbling window that outputs the number of message collected every 10 minutes.
-    // also emit early results if either the number of messages collected reaches 30000, or if no new messages arrive
-    // for 1 minute.
-    inputStream
-        .window(Windows.tumblingWindow(Duration.ofMinutes(10), initialValue, counter, new IntegerSerde())
-            .setLateTrigger(Triggers.any(Triggers.count(30000),
-                Triggers.timeSinceLastMessage(Duration.ofMinutes(1)))), "window")
-        .map(WindowPane::getMessage)
-        .sendTo(outputStream);
-  }
-
-  // local execution mode
-  public static void main(String[] args) throws Exception {
-    CommandLine cmdLine = new CommandLine();
-    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
-    LocalApplicationRunner localRunner = new LocalApplicationRunner(config);
-    localRunner.run(new WindowExample());
-  }
-
-  class PageViewEvent {
-    String key;
-    long timestamp;
-
-    public PageViewEvent(String key, long timestamp) {
-      this.key = key;
-      this.timestamp = timestamp;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/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 664f3b1..83fe5ad 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
@@ -34,8 +34,8 @@ import org.apache.samza.config.MapConfig;
 import org.apache.samza.config.TaskConfig;
 import org.apache.samza.operators.KV;
 import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.StreamGraphSpec;
 import org.apache.samza.operators.OutputStream;
-import org.apache.samza.operators.StreamGraphImpl;
 import org.apache.samza.operators.functions.JoinFunction;
 import org.apache.samza.operators.windows.Windows;
 import org.apache.samza.runtime.ApplicationRunner;
@@ -97,24 +97,24 @@ public class TestExecutionPlanner {
     };
   }
 
-  private StreamGraphImpl createSimpleGraph() {
+  private StreamGraphSpec createSimpleGraph() {
     /**
      * a simple graph of partitionBy and map
      *
      * input1 -> partitionBy -> map -> output1
      *
      */
-    StreamGraphImpl streamGraph = new StreamGraphImpl(runner, config);
-    MessageStream<KV<Object, Object>> input1 = streamGraph.getInputStream("input1");
-    OutputStream<KV<Object, Object>> output1 = streamGraph.getOutputStream("output1");
+    StreamGraphSpec graphSpec = new StreamGraphSpec(runner, config);
+    MessageStream<KV<Object, Object>> input1 = graphSpec.getInputStream("input1");
+    OutputStream<KV<Object, Object>> output1 = graphSpec.getOutputStream("output1");
     input1
         .partitionBy(m -> m.key, m -> m.value, "p1")
         .map(kv -> kv)
         .sendTo(output1);
-    return streamGraph;
+    return graphSpec;
   }
 
-  private StreamGraphImpl createStreamGraphWithJoin() {
+  private StreamGraphSpec createStreamGraphWithJoin() {
 
     /**
      * the graph looks like the following. number of partitions in parentheses. quotes indicate expected value.
@@ -127,76 +127,79 @@ public class TestExecutionPlanner {
      *
      */
 
-    StreamGraphImpl streamGraph = new StreamGraphImpl(runner, config);
+    StreamGraphSpec graphSpec = new StreamGraphSpec(runner, config);
     MessageStream<KV<Object, Object>> messageStream1 =
-        streamGraph.<KV<Object, Object>>getInputStream("input1")
+        graphSpec.<KV<Object, Object>>getInputStream("input1")
             .map(m -> m);
     MessageStream<KV<Object, Object>> messageStream2 =
-        streamGraph.<KV<Object, Object>>getInputStream("input2")
+        graphSpec.<KV<Object, Object>>getInputStream("input2")
             .partitionBy(m -> m.key, m -> m.value, "p1")
             .filter(m -> true);
     MessageStream<KV<Object, Object>> messageStream3 =
-        streamGraph.<KV<Object, Object>>getInputStream("input3")
+        graphSpec.<KV<Object, Object>>getInputStream("input3")
             .filter(m -> true)
             .partitionBy(m -> m.key, m -> m.value, "p2")
             .map(m -> m);
-    OutputStream<KV<Object, Object>> output1 = streamGraph.getOutputStream("output1");
-    OutputStream<KV<Object, Object>> output2 = streamGraph.getOutputStream("output2");
+    OutputStream<KV<Object, Object>> output1 = graphSpec.getOutputStream("output1");
+    OutputStream<KV<Object, Object>> output2 = graphSpec.getOutputStream("output2");
 
     messageStream1
-        .join(messageStream2, mock(JoinFunction.class),
+        .join(messageStream2,
+            (JoinFunction<Object, KV<Object, Object>, KV<Object, Object>, KV<Object, Object>>) mock(JoinFunction.class),
             mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(2), "j1")
         .sendTo(output1);
     messageStream3
-        .join(messageStream2, mock(JoinFunction.class),
+        .join(messageStream2,
+            (JoinFunction<Object, KV<Object, Object>, KV<Object, Object>, KV<Object, Object>>) mock(JoinFunction.class),
             mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(1), "j2")
         .sendTo(output2);
 
-    return streamGraph;
+    return graphSpec;
   }
 
-  private StreamGraphImpl createStreamGraphWithJoinAndWindow() {
+  private StreamGraphSpec createStreamGraphWithJoinAndWindow() {
 
-    StreamGraphImpl streamGraph = new StreamGraphImpl(runner, config);
+    StreamGraphSpec graphSpec = new StreamGraphSpec(runner, config);
     MessageStream<KV<Object, Object>> messageStream1 =
-        streamGraph.<KV<Object, Object>>getInputStream("input1")
+        graphSpec.<KV<Object, Object>>getInputStream("input1")
             .map(m -> m);
     MessageStream<KV<Object, Object>> messageStream2 =
-        streamGraph.<KV<Object, Object>>getInputStream("input2")
+        graphSpec.<KV<Object, Object>>getInputStream("input2")
             .partitionBy(m -> m.key, m -> m.value, "p1")
             .filter(m -> true);
     MessageStream<KV<Object, Object>> messageStream3 =
-        streamGraph.<KV<Object, Object>>getInputStream("input3")
+        graphSpec.<KV<Object, Object>>getInputStream("input3")
             .filter(m -> true)
             .partitionBy(m -> m.key, m -> m.value, "p2")
             .map(m -> m);
-    OutputStream<KV<Object, Object>> output1 = streamGraph.getOutputStream("output1");
-    OutputStream<KV<Object, Object>> output2 = streamGraph.getOutputStream("output2");
+    OutputStream<KV<Object, Object>> output1 = graphSpec.getOutputStream("output1");
+    OutputStream<KV<Object, Object>> output2 = graphSpec.getOutputStream("output2");
 
     messageStream1.map(m -> m)
         .filter(m->true)
-        .window(Windows.<KV<Object, Object>, Object>keyedTumblingWindow(m -> m, Duration.ofMillis(8),
-            mock(Serde.class), mock(Serde.class)), "w1");
+        .window(Windows.keyedTumblingWindow(m -> m, Duration.ofMillis(8), mock(Serde.class), mock(Serde.class)), "w1");
 
     messageStream2.map(m -> m)
         .filter(m->true)
-        .window(Windows.<KV<Object, Object>, Object>keyedTumblingWindow(m -> m, Duration.ofMillis(16),
-            mock(Serde.class), mock(Serde.class)), "w2");
+        .window(Windows.keyedTumblingWindow(m -> m, Duration.ofMillis(16), mock(Serde.class), mock(Serde.class)), "w2");
 
     messageStream1
-        .join(messageStream2, mock(JoinFunction.class),
+        .join(messageStream2,
+            (JoinFunction<Object, KV<Object, Object>, KV<Object, Object>, KV<Object, Object>>) mock(JoinFunction.class),
             mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofMillis(1600), "j1")
         .sendTo(output1);
     messageStream3
-        .join(messageStream2, mock(JoinFunction.class),
+        .join(messageStream2,
+            (JoinFunction<Object, KV<Object, Object>, KV<Object, Object>, KV<Object, Object>>) mock(JoinFunction.class),
             mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofMillis(100), "j2")
         .sendTo(output2);
     messageStream3
-        .join(messageStream2, mock(JoinFunction.class),
+        .join(messageStream2,
+            (JoinFunction<Object, KV<Object, Object>, KV<Object, Object>, KV<Object, Object>>) mock(JoinFunction.class),
             mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofMillis(252), "j3")
         .sendTo(output2);
 
-    return streamGraph;
+    return graphSpec;
   }
 
   @Before
@@ -252,9 +255,9 @@ public class TestExecutionPlanner {
   @Test
   public void testCreateProcessorGraph() {
     ExecutionPlanner planner = new ExecutionPlanner(config, streamManager);
-    StreamGraphImpl streamGraph = createStreamGraphWithJoin();
+    StreamGraphSpec graphSpec = createStreamGraphWithJoin();
 
-    JobGraph jobGraph = planner.createJobGraph(streamGraph);
+    JobGraph jobGraph = planner.createJobGraph(graphSpec.getOperatorSpecGraph());
     assertTrue(jobGraph.getSources().size() == 3);
     assertTrue(jobGraph.getSinks().size() == 2);
     assertTrue(jobGraph.getIntermediateStreams().size() == 2); // two streams generated by partitionBy
@@ -263,8 +266,8 @@ public class TestExecutionPlanner {
   @Test
   public void testFetchExistingStreamPartitions() {
     ExecutionPlanner planner = new ExecutionPlanner(config, streamManager);
-    StreamGraphImpl streamGraph = createStreamGraphWithJoin();
-    JobGraph jobGraph = planner.createJobGraph(streamGraph);
+    StreamGraphSpec graphSpec = createStreamGraphWithJoin();
+    JobGraph jobGraph = planner.createJobGraph(graphSpec.getOperatorSpecGraph());
 
     ExecutionPlanner.updateExistingPartitions(jobGraph, streamManager);
     assertTrue(jobGraph.getOrCreateStreamEdge(input1).getPartitionCount() == 64);
@@ -281,11 +284,11 @@ public class TestExecutionPlanner {
   @Test
   public void testCalculateJoinInputPartitions() {
     ExecutionPlanner planner = new ExecutionPlanner(config, streamManager);
-    StreamGraphImpl streamGraph = createStreamGraphWithJoin();
-    JobGraph jobGraph = planner.createJobGraph(streamGraph);
+    StreamGraphSpec graphSpec = createStreamGraphWithJoin();
+    JobGraph jobGraph = planner.createJobGraph(graphSpec.getOperatorSpecGraph());
 
     ExecutionPlanner.updateExistingPartitions(jobGraph, streamManager);
-    ExecutionPlanner.calculateJoinInputPartitions(streamGraph, jobGraph);
+    ExecutionPlanner.calculateJoinInputPartitions(jobGraph);
 
     // the partitions should be the same as input1
     jobGraph.getIntermediateStreams().forEach(edge -> {
@@ -300,9 +303,9 @@ public class TestExecutionPlanner {
     Config cfg = new MapConfig(map);
 
     ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager);
-    StreamGraphImpl streamGraph = createSimpleGraph();
-    JobGraph jobGraph = planner.createJobGraph(streamGraph);
-    planner.calculatePartitions(streamGraph, jobGraph);
+    StreamGraphSpec graphSpec = createSimpleGraph();
+    JobGraph jobGraph = planner.createJobGraph(graphSpec.getOperatorSpecGraph());
+    planner.calculatePartitions(jobGraph);
 
     // the partitions should be the same as input1
     jobGraph.getIntermediateStreams().forEach(edge -> {
@@ -317,8 +320,8 @@ public class TestExecutionPlanner {
     Config cfg = new MapConfig(map);
 
     ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager);
-    StreamGraphImpl streamGraph = createStreamGraphWithJoin();
-    ExecutionPlan plan = planner.plan(streamGraph);
+    StreamGraphSpec graphSpec = createStreamGraphWithJoin();
+    ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph());
     List<JobConfig> jobConfigs = plan.getJobConfigs();
     for (JobConfig config : jobConfigs) {
       System.out.println(config);
@@ -332,8 +335,8 @@ public class TestExecutionPlanner {
     Config cfg = new MapConfig(map);
 
     ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager);
-    StreamGraphImpl streamGraph = createStreamGraphWithJoinAndWindow();
-    ExecutionPlan plan = planner.plan(streamGraph);
+    StreamGraphSpec graphSpec = createStreamGraphWithJoinAndWindow();
+    ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph());
     List<JobConfig> jobConfigs = plan.getJobConfigs();
     assertEquals(1, jobConfigs.size());
 
@@ -349,8 +352,8 @@ public class TestExecutionPlanner {
     Config cfg = new MapConfig(map);
 
     ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager);
-    StreamGraphImpl streamGraph = createStreamGraphWithJoinAndWindow();
-    ExecutionPlan plan = planner.plan(streamGraph);
+    StreamGraphSpec graphSpec = createStreamGraphWithJoinAndWindow();
+    ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph());
     List<JobConfig> jobConfigs = plan.getJobConfigs();
     assertEquals(1, jobConfigs.size());
 
@@ -366,8 +369,8 @@ public class TestExecutionPlanner {
     Config cfg = new MapConfig(map);
 
     ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager);
-    StreamGraphImpl streamGraph = createSimpleGraph();
-    ExecutionPlan plan = planner.plan(streamGraph);
+    StreamGraphSpec graphSpec = createSimpleGraph();
+    ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph());
     List<JobConfig> jobConfigs = plan.getJobConfigs();
     assertEquals(1, jobConfigs.size());
     assertFalse(jobConfigs.get(0).containsKey(TaskConfig.WINDOW_MS()));
@@ -381,8 +384,8 @@ public class TestExecutionPlanner {
     Config cfg = new MapConfig(map);
 
     ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager);
-    StreamGraphImpl streamGraph = createSimpleGraph();
-    ExecutionPlan plan = planner.plan(streamGraph);
+    StreamGraphSpec graphSpec = createSimpleGraph();
+    ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph());
     List<JobConfig> jobConfigs = plan.getJobConfigs();
     assertEquals(1, jobConfigs.size());
     assertEquals("2000", jobConfigs.get(0).get(TaskConfig.WINDOW_MS()));
@@ -391,8 +394,8 @@ public class TestExecutionPlanner {
   @Test
   public void testCalculateIntStreamPartitions() throws Exception {
     ExecutionPlanner planner = new ExecutionPlanner(config, streamManager);
-    StreamGraphImpl streamGraph = createSimpleGraph();
-    JobGraph jobGraph = (JobGraph) planner.plan(streamGraph);
+    StreamGraphSpec graphSpec = createSimpleGraph();
+    JobGraph jobGraph = (JobGraph) planner.plan(graphSpec.getOperatorSpecGraph());
 
     // the partitions should be the same as input1
     jobGraph.getIntermediateStreams().forEach(edge -> {
@@ -424,12 +427,12 @@ public class TestExecutionPlanner {
     int partitionLimit = ExecutionPlanner.MAX_INFERRED_PARTITIONS;
 
     ExecutionPlanner planner = new ExecutionPlanner(config, streamManager);
-    StreamGraphImpl streamGraph = new StreamGraphImpl(runner, config);
+    StreamGraphSpec graphSpec = new StreamGraphSpec(runner, config);
 
-    MessageStream<KV<Object, Object>> input1 = streamGraph.getInputStream("input4");
-    OutputStream<KV<Object, Object>> output1 = streamGraph.getOutputStream("output1");
+    MessageStream<KV<Object, Object>> input1 = graphSpec.getInputStream("input4");
+    OutputStream<KV<Object, Object>> output1 = graphSpec.getOutputStream("output1");
     input1.partitionBy(m -> m.key, m -> m.value, "p1").map(kv -> kv).sendTo(output1);
-    JobGraph jobGraph = (JobGraph) planner.plan(streamGraph);
+    JobGraph jobGraph = (JobGraph) planner.plan(graphSpec.getOperatorSpecGraph());
 
     // the partitions should be the same as input1
     jobGraph.getIntermediateStreams().forEach(edge -> {

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java
index bf131ce..359c422 100644
--- a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java
+++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraph.java
@@ -57,16 +57,16 @@ public class TestJobGraph {
    * 2 9 10
    */
   private void createGraph1() {
-    graph1 = new JobGraph(null);
+    graph1 = new JobGraph(null, null);
 
-    JobNode n2 = graph1.getOrCreateJobNode("2", "1", null);
-    JobNode n3 = graph1.getOrCreateJobNode("3", "1", null);
-    JobNode n5 = graph1.getOrCreateJobNode("5", "1", null);
-    JobNode n7 = graph1.getOrCreateJobNode("7", "1", null);
-    JobNode n8 = graph1.getOrCreateJobNode("8", "1", null);
-    JobNode n9 = graph1.getOrCreateJobNode("9", "1", null);
-    JobNode n10 = graph1.getOrCreateJobNode("10", "1", null);
-    JobNode n11 = graph1.getOrCreateJobNode("11", "1", null);
+    JobNode n2 = graph1.getOrCreateJobNode("2", "1");
+    JobNode n3 = graph1.getOrCreateJobNode("3", "1");
+    JobNode n5 = graph1.getOrCreateJobNode("5", "1");
+    JobNode n7 = graph1.getOrCreateJobNode("7", "1");
+    JobNode n8 = graph1.getOrCreateJobNode("8", "1");
+    JobNode n9 = graph1.getOrCreateJobNode("9", "1");
+    JobNode n10 = graph1.getOrCreateJobNode("10", "1");
+    JobNode n11 = graph1.getOrCreateJobNode("11", "1");
 
     graph1.addSource(genStream(), n5);
     graph1.addSource(genStream(), n7);
@@ -90,15 +90,15 @@ public class TestJobGraph {
    *      |<---6 <--|    <>
    */
   private void createGraph2() {
-    graph2 = new JobGraph(null);
+    graph2 = new JobGraph(null, null);
 
-    JobNode n1 = graph2.getOrCreateJobNode("1", "1", null);
-    JobNode n2 = graph2.getOrCreateJobNode("2", "1", null);
-    JobNode n3 = graph2.getOrCreateJobNode("3", "1", null);
-    JobNode n4 = graph2.getOrCreateJobNode("4", "1", null);
-    JobNode n5 = graph2.getOrCreateJobNode("5", "1", null);
-    JobNode n6 = graph2.getOrCreateJobNode("6", "1", null);
-    JobNode n7 = graph2.getOrCreateJobNode("7", "1", null);
+    JobNode n1 = graph2.getOrCreateJobNode("1", "1");
+    JobNode n2 = graph2.getOrCreateJobNode("2", "1");
+    JobNode n3 = graph2.getOrCreateJobNode("3", "1");
+    JobNode n4 = graph2.getOrCreateJobNode("4", "1");
+    JobNode n5 = graph2.getOrCreateJobNode("5", "1");
+    JobNode n6 = graph2.getOrCreateJobNode("6", "1");
+    JobNode n7 = graph2.getOrCreateJobNode("7", "1");
 
     graph2.addSource(genStream(), n1);
     graph2.addIntermediateStream(genStream(), n1, n2);
@@ -117,10 +117,10 @@ public class TestJobGraph {
    * 1<->1 -> 2<->2
    */
   private void createGraph3() {
-    graph3 = new JobGraph(null);
+    graph3 = new JobGraph(null, null);
 
-    JobNode n1 = graph3.getOrCreateJobNode("1", "1", null);
-    JobNode n2 = graph3.getOrCreateJobNode("2", "1", null);
+    JobNode n1 = graph3.getOrCreateJobNode("1", "1");
+    JobNode n2 = graph3.getOrCreateJobNode("2", "1");
 
     graph3.addSource(genStream(), n1);
     graph3.addIntermediateStream(genStream(), n1, n1);
@@ -133,9 +133,9 @@ public class TestJobGraph {
    * 1<->1
    */
   private void createGraph4() {
-    graph4 = new JobGraph(null);
+    graph4 = new JobGraph(null, null);
 
-    JobNode n1 = graph4.getOrCreateJobNode("1", "1", null);
+    JobNode n1 = graph4.getOrCreateJobNode("1", "1");
 
     graph4.addSource(genStream(), n1);
     graph4.addIntermediateStream(genStream(), n1, n1);
@@ -151,7 +151,7 @@ public class TestJobGraph {
 
   @Test
   public void testAddSource() {
-    JobGraph graph = new JobGraph(null);
+    JobGraph graph = new JobGraph(null, null);
 
     /**
      * s1 -> 1
@@ -160,9 +160,9 @@ public class TestJobGraph {
      * s3 -> 2
      *   |-> 3
      */
-    JobNode n1 = graph.getOrCreateJobNode("1", "1", null);
-    JobNode n2 = graph.getOrCreateJobNode("2", "1", null);
-    JobNode n3 = graph.getOrCreateJobNode("3", "1", null);
+    JobNode n1 = graph.getOrCreateJobNode("1", "1");
+    JobNode n2 = graph.getOrCreateJobNode("2", "1");
+    JobNode n3 = graph.getOrCreateJobNode("3", "1");
     StreamSpec s1 = genStream();
     StreamSpec s2 = genStream();
     StreamSpec s3 = genStream();
@@ -173,9 +173,9 @@ public class TestJobGraph {
 
     assertTrue(graph.getSources().size() == 3);
 
-    assertTrue(graph.getOrCreateJobNode("1", "1", null).getInEdges().size() == 2);
-    assertTrue(graph.getOrCreateJobNode("2", "1", null).getInEdges().size() == 1);
-    assertTrue(graph.getOrCreateJobNode("3", "1", null).getInEdges().size() == 1);
+    assertTrue(graph.getOrCreateJobNode("1", "1").getInEdges().size() == 2);
+    assertTrue(graph.getOrCreateJobNode("2", "1").getInEdges().size() == 1);
+    assertTrue(graph.getOrCreateJobNode("3", "1").getInEdges().size() == 1);
 
     assertTrue(graph.getOrCreateStreamEdge(s1).getSourceNodes().size() == 0);
     assertTrue(graph.getOrCreateStreamEdge(s1).getTargetNodes().size() == 1);
@@ -192,9 +192,9 @@ public class TestJobGraph {
      * 2 -> s2
      * 2 -> s3
      */
-    JobGraph graph = new JobGraph(null);
-    JobNode n1 = graph.getOrCreateJobNode("1", "1", null);
-    JobNode n2 = graph.getOrCreateJobNode("2", "1", null);
+    JobGraph graph = new JobGraph(null, null);
+    JobNode n1 = graph.getOrCreateJobNode("1", "1");
+    JobNode n2 = graph.getOrCreateJobNode("2", "1");
     StreamSpec s1 = genStream();
     StreamSpec s2 = genStream();
     StreamSpec s3 = genStream();
@@ -203,8 +203,8 @@ public class TestJobGraph {
     graph.addSink(s3, n2);
 
     assertTrue(graph.getSinks().size() == 3);
-    assertTrue(graph.getOrCreateJobNode("1", "1", null).getOutEdges().size() == 1);
-    assertTrue(graph.getOrCreateJobNode("2", "1", null).getOutEdges().size() == 2);
+    assertTrue(graph.getOrCreateJobNode("1", "1").getOutEdges().size() == 1);
+    assertTrue(graph.getOrCreateJobNode("2", "1").getOutEdges().size() == 2);
 
     assertTrue(graph.getOrCreateStreamEdge(s1).getSourceNodes().size() == 1);
     assertTrue(graph.getOrCreateStreamEdge(s1).getTargetNodes().size() == 0);

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java
index f218e89..abe8969 100644
--- a/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java
+++ b/samza-core/src/test/java/org/apache/samza/execution/TestJobGraphJsonGenerator.java
@@ -28,7 +28,7 @@ import org.apache.samza.config.MapConfig;
 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.operators.StreamGraphSpec;
 import org.apache.samza.operators.functions.JoinFunction;
 import org.apache.samza.operators.windows.Windows;
 import org.apache.samza.runtime.ApplicationRunner;
@@ -114,35 +114,37 @@ public class TestJobGraphJsonGenerator {
     when(systemAdmins.getSystemAdmin("system2")).thenReturn(systemAdmin2);
     StreamManager streamManager = new StreamManager(systemAdmins);
 
-    StreamGraphImpl streamGraph = new StreamGraphImpl(runner, config);
-    streamGraph.setDefaultSerde(KVSerde.of(new NoOpSerde<>(), new NoOpSerde<>()));
+    StreamGraphSpec graphSpec = new StreamGraphSpec(runner, config);
+    graphSpec.setDefaultSerde(KVSerde.of(new NoOpSerde<>(), new NoOpSerde<>()));
     MessageStream<KV<Object, Object>> messageStream1 =
-        streamGraph.<KV<Object, Object>>getInputStream("input1")
+        graphSpec.<KV<Object, Object>>getInputStream("input1")
             .map(m -> m);
     MessageStream<KV<Object, Object>> messageStream2 =
-        streamGraph.<KV<Object, Object>>getInputStream("input2")
+        graphSpec.<KV<Object, Object>>getInputStream("input2")
             .partitionBy(m -> m.key, m -> m.value, "p1")
             .filter(m -> true);
     MessageStream<KV<Object, Object>> messageStream3 =
-        streamGraph.<KV<Object, Object>>getInputStream("input3")
+        graphSpec.<KV<Object, Object>>getInputStream("input3")
             .filter(m -> true)
             .partitionBy(m -> m.key, m -> m.value, "p2")
             .map(m -> m);
-    OutputStream<KV<Object, Object>> outputStream1 = streamGraph.getOutputStream("output1");
-    OutputStream<KV<Object, Object>> outputStream2 = streamGraph.getOutputStream("output2");
+    OutputStream<KV<Object, Object>> outputStream1 = graphSpec.getOutputStream("output1");
+    OutputStream<KV<Object, Object>> outputStream2 = graphSpec.getOutputStream("output2");
 
     messageStream1
-        .join(messageStream2, mock(JoinFunction.class),
+        .join(messageStream2,
+            (JoinFunction<Object, KV<Object, Object>, KV<Object, Object>, KV<Object, Object>>) mock(JoinFunction.class),
             mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(2), "j1")
         .sendTo(outputStream1);
     messageStream2.sink((message, collector, coordinator) -> { });
     messageStream3
-        .join(messageStream2, mock(JoinFunction.class),
+        .join(messageStream2,
+            (JoinFunction<Object, KV<Object, Object>, KV<Object, Object>, KV<Object, Object>>) mock(JoinFunction.class),
             mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofHours(1), "j2")
         .sendTo(outputStream2);
 
     ExecutionPlanner planner = new ExecutionPlanner(config, streamManager);
-    ExecutionPlan plan = planner.plan(streamGraph);
+    ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph());
     String json = plan.getPlanAsJson();
     System.out.println(json);
 
@@ -187,8 +189,8 @@ public class TestJobGraphJsonGenerator {
     when(systemAdmins.getSystemAdmin("kafka")).thenReturn(systemAdmin2);
     StreamManager streamManager = new StreamManager(systemAdmins);
 
-    StreamGraphImpl streamGraph = new StreamGraphImpl(runner, config);
-    MessageStream<KV<String, PageViewEvent>> inputStream = streamGraph.getInputStream("PageView");
+    StreamGraphSpec graphSpec = new StreamGraphSpec(runner, config);
+    MessageStream<KV<String, PageViewEvent>> inputStream = graphSpec.getInputStream("PageView");
     inputStream
         .partitionBy(kv -> kv.getValue().getCountry(), kv -> kv.getValue(), "keyed-by-country")
         .window(Windows.keyedTumblingWindow(kv -> kv.getValue().getCountry(),
@@ -198,10 +200,10 @@ public class TestJobGraphJsonGenerator {
             new StringSerde(),
             new LongSerde()), "count-by-country")
         .map(pane -> new KV<>(pane.getKey().getKey(), pane.getMessage()))
-        .sendTo(streamGraph.getOutputStream("PageViewCount"));
+        .sendTo(graphSpec.getOutputStream("PageViewCount"));
 
     ExecutionPlanner planner = new ExecutionPlanner(config, streamManager);
-    ExecutionPlan plan = planner.plan(streamGraph);
+    ExecutionPlan plan = planner.plan(graphSpec.getOperatorSpecGraph());
     String json = plan.getPlanAsJson();
     System.out.println(json);
 

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java b/samza-core/src/test/java/org/apache/samza/execution/TestJobNode.java
index 53e8bf6..c43e242 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
@@ -25,8 +25,8 @@ 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.StreamGraphSpec;
 import org.apache.samza.operators.OutputStream;
-import org.apache.samza.operators.StreamGraphImpl;
 import org.apache.samza.operators.functions.JoinFunction;
 import org.apache.samza.operators.impl.store.TimestampedValueSerde;
 import org.apache.samza.runtime.ApplicationRunner;
@@ -71,11 +71,11 @@ public class TestJobNode {
     when(mockConfig.get(JobConfig.JOB_NAME())).thenReturn("jobName");
     when(mockConfig.get(eq(JobConfig.JOB_ID()), anyString())).thenReturn("jobId");
 
-    StreamGraphImpl streamGraph = new StreamGraphImpl(mockRunner, mockConfig);
-    streamGraph.setDefaultSerde(KVSerde.of(new StringSerde(), new JsonSerdeV2<>()));
-    MessageStream<KV<String, Object>> input1 = streamGraph.getInputStream("input1");
-    MessageStream<KV<String, Object>> input2 = streamGraph.getInputStream("input2");
-    OutputStream<KV<String, Object>> output = streamGraph.getOutputStream("output");
+    StreamGraphSpec graphSpec = new StreamGraphSpec(mockRunner, mockConfig);
+    graphSpec.setDefaultSerde(KVSerde.of(new StringSerde(), new JsonSerdeV2<>()));
+    MessageStream<KV<String, Object>> input1 = graphSpec.getInputStream("input1");
+    MessageStream<KV<String, Object>> input2 = graphSpec.getInputStream("input2");
+    OutputStream<KV<String, Object>> output = graphSpec.getOutputStream("output");
     JoinFunction<String, Object, Object, KV<String, Object>> mockJoinFn = mock(JoinFunction.class);
     input1
         .partitionBy(KV::getKey, KV::getValue, "p1").map(kv -> kv.value)
@@ -84,7 +84,7 @@ public class TestJobNode {
             Duration.ofHours(1), "j1")
         .sendTo(output);
 
-    JobNode jobNode = new JobNode("jobName", "jobId", streamGraph, mockConfig);
+    JobNode jobNode = new JobNode("jobName", "jobId", graphSpec.getOperatorSpecGraph(), mockConfig);
     Config config = new MapConfig();
     StreamEdge input1Edge = new StreamEdge(input1Spec, config);
     StreamEdge input2Edge = new StreamEdge(input2Spec, config);


[03/10] samza git commit: SAMZA-1659: Serializable OperatorSpec

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestSamzaSqlQueryParser.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestSamzaSqlQueryParser.java b/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestSamzaSqlQueryParser.java
new file mode 100644
index 0000000..be1e317
--- /dev/null
+++ b/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestSamzaSqlQueryParser.java
@@ -0,0 +1,75 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*   http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.sql.testutil;
+
+import org.apache.samza.SamzaException;
+import org.apache.samza.sql.testutil.SamzaSqlQueryParser.QueryInfo;
+import org.junit.Test;
+
+import junit.framework.Assert;
+
+public class TestSamzaSqlQueryParser {
+
+  @Test
+  public void testParseQuery() {
+    QueryInfo queryInfo = SamzaSqlQueryParser.parseQuery("insert into log.foo select * from tracking.bar");
+    Assert.assertEquals("log.foo", queryInfo.getSink());
+    Assert.assertEquals(queryInfo.getSelectQuery(), "select * from tracking.bar", queryInfo.getSelectQuery());
+    Assert.assertEquals(1, queryInfo.getSources().size());
+    Assert.assertEquals("tracking.bar", queryInfo.getSources().get(0));
+  }
+
+  @Test
+  public void testParseJoinQuery() {
+    String sql =
+        "Insert into testavro.enrichedPageViewTopic"
+            + " select p.name as profileName, pv.pageKey"
+            + " from testavro.PAGEVIEW as pv"
+            + " join testavro.PROFILE.`$table` as p"
+            + " on p.id = pv.profileId";
+    QueryInfo queryInfo = SamzaSqlQueryParser.parseQuery(sql);
+    Assert.assertEquals("testavro.enrichedPageViewTopic", queryInfo.getSink());
+    Assert.assertEquals(2, queryInfo.getSources().size());
+    Assert.assertEquals("testavro.PAGEVIEW", queryInfo.getSources().get(0));
+    Assert.assertEquals("testavro.PROFILE.$table", queryInfo.getSources().get(1));
+  }
+
+  @Test
+  public void testParseInvalidQuery() {
+
+    try {
+      SamzaSqlQueryParser.parseQuery("select * from tracking.bar");
+      Assert.fail("Expected a samzaException");
+    } catch (SamzaException e) {
+    }
+
+    try {
+      SamzaSqlQueryParser.parseQuery("insert into select * from tracking.bar");
+      Assert.fail("Expected a samzaException");
+    } catch (SamzaException e) {
+    }
+
+    try {
+      SamzaSqlQueryParser.parseQuery("insert into log.off select from tracking.bar");
+      Assert.fail("Expected a samzaException");
+    } catch (SamzaException e) {
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java
new file mode 100644
index 0000000..88ce443
--- /dev/null
+++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java
@@ -0,0 +1,136 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*   http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.sql.translator;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.samza.config.Config;
+import org.apache.samza.container.TaskContextImpl;
+import org.apache.samza.container.TaskName;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.StreamGraphSpec;
+import org.apache.samza.operators.functions.FilterFunction;
+import org.apache.samza.operators.spec.OperatorSpec;
+import org.apache.samza.operators.spec.StreamOperatorSpec;
+import org.apache.samza.sql.data.Expression;
+import org.apache.samza.sql.data.RexToJavaCompiler;
+import org.apache.samza.sql.data.SamzaSqlExecutionContext;
+import org.apache.samza.sql.data.SamzaSqlRelMessage;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.internal.util.reflection.Whitebox;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for {@link FilterTranslator}
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest(LogicalFilter.class)
+public class TestFilterTranslator extends TranslatorTestBase {
+
+  @Test
+  public void testTranslate() throws IOException, ClassNotFoundException {
+    // setup mock values to the constructor of FilterTranslator
+    LogicalFilter mockFilter = PowerMockito.mock(LogicalFilter.class);
+    TranslatorContext mockContext = mock(TranslatorContext.class);
+    RelNode mockInput = mock(RelNode.class);
+    when(mockFilter.getInput()).thenReturn(mockInput);
+    when(mockInput.getId()).thenReturn(1);
+    when(mockFilter.getId()).thenReturn(2);
+    StreamGraphSpec mockGraph = mock(StreamGraphSpec.class);
+    OperatorSpec<Object, SamzaSqlRelMessage> mockInputOp = mock(OperatorSpec.class);
+    MessageStream<SamzaSqlRelMessage> mockStream = new MessageStreamImpl<>(mockGraph, mockInputOp);
+    when(mockContext.getMessageStream(eq(1))).thenReturn(mockStream);
+    doAnswer(this.getRegisterMessageStreamAnswer()).when(mockContext).registerMessageStream(eq(2), any(MessageStream.class));
+    RexToJavaCompiler mockCompiler = mock(RexToJavaCompiler.class);
+    when(mockContext.getExpressionCompiler()).thenReturn(mockCompiler);
+    Expression mockExpr = mock(Expression.class);
+    when(mockCompiler.compile(any(), any())).thenReturn(mockExpr);
+
+    // Apply translate() method to verify that we are getting the correct filter operator constructed
+    FilterTranslator filterTranslator = new FilterTranslator();
+    filterTranslator.translate(mockFilter, mockContext);
+    // make sure that context has been registered with LogicFilter and output message streams
+    verify(mockContext, times(1)).registerRelNode(2, mockFilter);
+    verify(mockContext, times(1)).registerMessageStream(2, this.getRegisteredMessageStream(2));
+    when(mockContext.getRelNode(2)).thenReturn(mockFilter);
+    when(mockContext.getMessageStream(2)).thenReturn(this.getRegisteredMessageStream(2));
+    StreamOperatorSpec filterSpec = (StreamOperatorSpec) Whitebox.getInternalState(this.getRegisteredMessageStream(2), "operatorSpec");
+    assertNotNull(filterSpec);
+    assertEquals(filterSpec.getOpCode(), OperatorSpec.OpCode.FILTER);
+
+    // Verify that the init() method will establish the context for the filter function
+    Config mockConfig = mock(Config.class);
+    TaskContextImpl taskContext = new TaskContextImpl(new TaskName("Partition-1"), null, null,
+        new HashSet<>(), null, null, null, null, null, null);
+    taskContext.setUserContext(mockContext);
+    filterSpec.getTransformFn().init(mockConfig, taskContext);
+    FilterFunction filterFn = (FilterFunction) Whitebox.getInternalState(filterSpec, "filterFn");
+    assertNotNull(filterFn);
+    assertEquals(mockContext, Whitebox.getInternalState(filterFn, "context"));
+    assertEquals(mockFilter, Whitebox.getInternalState(filterFn, "filter"));
+    assertEquals(mockExpr, Whitebox.getInternalState(filterFn, "expr"));
+
+    // Calling filterFn.apply() to verify the filter function is correctly applied to the input message
+    SamzaSqlRelMessage mockInputMsg = new SamzaSqlRelMessage(new ArrayList<>(), new ArrayList<>());
+    SamzaSqlExecutionContext executionContext = mock(SamzaSqlExecutionContext.class);
+    DataContext dataContext = mock(DataContext.class);
+    when(mockContext.getExecutionContext()).thenReturn(executionContext);
+    when(mockContext.getDataContext()).thenReturn(dataContext);
+    Object[] result = new Object[1];
+
+    doAnswer( invocation -> {
+      Object[] retValue = invocation.getArgumentAt(3, Object[].class);
+      retValue[0] = new Boolean(true);
+      return null;
+    }).when(mockExpr).execute(eq(executionContext), eq(dataContext),
+        eq(mockInputMsg.getSamzaSqlRelRecord().getFieldValues().toArray()), eq(result));
+    assertTrue(filterFn.apply(mockInputMsg));
+
+    doAnswer( invocation -> {
+      Object[] retValue = invocation.getArgumentAt(3, Object[].class);
+      retValue[0] = new Boolean(false);
+      return null;
+    }).when(mockExpr).execute(eq(executionContext), eq(dataContext),
+        eq(mockInputMsg.getSamzaSqlRelRecord().getFieldValues().toArray()), eq(result));
+    assertFalse(filterFn.apply(mockInputMsg));
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/test/java/org/apache/samza/sql/translator/TestJoinTranslator.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestJoinTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestJoinTranslator.java
new file mode 100644
index 0000000..2de4856
--- /dev/null
+++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestJoinTranslator.java
@@ -0,0 +1,191 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*   http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.sql.translator;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.StreamGraphSpec;
+import org.apache.samza.operators.TableDescriptor;
+import org.apache.samza.operators.functions.StreamTableJoinFunction;
+import org.apache.samza.operators.spec.InputOperatorSpec;
+import org.apache.samza.operators.spec.OperatorSpec;
+import org.apache.samza.operators.spec.OutputStreamImpl;
+import org.apache.samza.operators.spec.StreamTableJoinOperatorSpec;
+import org.apache.samza.operators.stream.IntermediateMessageStreamImpl;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.sql.data.Expression;
+import org.apache.samza.sql.data.RexToJavaCompiler;
+import org.apache.samza.sql.data.SamzaSqlRelMessage;
+import org.apache.samza.sql.interfaces.SqlIOConfig;
+import org.apache.samza.sql.interfaces.SqlIOResolver;
+import org.apache.samza.storage.kv.RocksDbTableDescriptor;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.internal.util.reflection.Whitebox;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for {@link JoinTranslator}
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({LogicalJoin.class, EnumerableTableScan.class})
+public class TestJoinTranslator extends TranslatorTestBase {
+
+  @Test
+  public void testTranslateStreamToTableJoin() throws IOException, ClassNotFoundException {
+    // setup mock values to the constructor of FilterTranslator
+    LogicalJoin mockJoin = PowerMockito.mock(LogicalJoin.class);
+    TranslatorContext mockContext = mock(TranslatorContext.class);
+    RelNode mockLeftInput = PowerMockito.mock(EnumerableTableScan.class);
+    RelNode mockRightInput = mock(RelNode.class);
+    List<RelNode> inputs = new ArrayList<>();
+    inputs.add(mockLeftInput);
+    inputs.add(mockRightInput);
+    RelOptTable mockLeftTable = mock(RelOptTable.class);
+    when(mockLeftInput.getTable()).thenReturn(mockLeftTable);
+    List<String> qualifiedTableName = new ArrayList<String>() {{
+      this.add("test");
+      this.add("LeftTable");
+    }};
+    when(mockLeftTable.getQualifiedName()).thenReturn(qualifiedTableName);
+    when(mockLeftInput.getId()).thenReturn(1);
+    when(mockRightInput.getId()).thenReturn(2);
+    when(mockJoin.getId()).thenReturn(3);
+    when(mockJoin.getInputs()).thenReturn(inputs);
+    when(mockJoin.getLeft()).thenReturn(mockLeftInput);
+    when(mockJoin.getRight()).thenReturn(mockRightInput);
+    RexCall mockJoinCondition = mock(RexCall.class);
+    when(mockJoinCondition.isAlwaysTrue()).thenReturn(false);
+    when(mockJoinCondition.getKind()).thenReturn(SqlKind.EQUALS);
+    when(mockJoin.getCondition()).thenReturn(mockJoinCondition);
+    RexInputRef mockLeftConditionInput = mock(RexInputRef.class);
+    RexInputRef mockRightConditionInput = mock(RexInputRef.class);
+    when(mockLeftConditionInput.getIndex()).thenReturn(0);
+    when(mockRightConditionInput.getIndex()).thenReturn(0);
+    List<RexNode> condOperands = new ArrayList<>();
+    condOperands.add(mockLeftConditionInput);
+    condOperands.add(mockRightConditionInput);
+    when(mockJoinCondition.getOperands()).thenReturn(condOperands);
+    RelDataType mockLeftCondDataType = mock(RelDataType.class);
+    RelDataType mockRightCondDataType = mock(RelDataType.class);
+    when(mockLeftCondDataType.getSqlTypeName()).thenReturn(SqlTypeName.BOOLEAN);
+    when(mockRightCondDataType.getSqlTypeName()).thenReturn(SqlTypeName.BOOLEAN);
+    when(mockLeftConditionInput.getType()).thenReturn(mockLeftCondDataType);
+    when(mockRightConditionInput.getType()).thenReturn(mockRightCondDataType);
+    RelDataType mockLeftRowType = mock(RelDataType.class);
+    when(mockLeftRowType.getFieldCount()).thenReturn(0); //?? why ??
+
+    when(mockLeftInput.getRowType()).thenReturn(mockLeftRowType);
+    List<String> leftFieldNames = new ArrayList<String>() {{
+      this.add("test_table_field1");
+    }};
+    List<String> rightStreamFieldNames = new ArrayList<String>() {
+      {
+        this.add("test_stream_field1");
+      } };
+    when(mockLeftRowType.getFieldNames()).thenReturn(leftFieldNames);
+    RelDataType mockRightRowType = mock(RelDataType.class);
+    when(mockRightInput.getRowType()).thenReturn(mockRightRowType);
+    when(mockRightRowType.getFieldNames()).thenReturn(rightStreamFieldNames);
+
+    StreamGraphSpec mockGraph = mock(StreamGraphSpec.class);
+    OperatorSpec<Object, SamzaSqlRelMessage> mockLeftInputOp = mock(OperatorSpec.class);
+    MessageStream<SamzaSqlRelMessage> mockLeftInputStream = new MessageStreamImpl<>(mockGraph, mockLeftInputOp);
+    when(mockContext.getMessageStream(eq(mockLeftInput.getId()))).thenReturn(mockLeftInputStream);
+    OperatorSpec<Object, SamzaSqlRelMessage> mockRightInputOp = mock(OperatorSpec.class);
+    MessageStream<SamzaSqlRelMessage> mockRightInputStream = new MessageStreamImpl<>(mockGraph, mockRightInputOp);
+    when(mockContext.getMessageStream(eq(mockRightInput.getId()))).thenReturn(mockRightInputStream);
+    when(mockContext.getStreamGraph()).thenReturn(mockGraph);
+
+    InputOperatorSpec mockInputOp = mock(InputOperatorSpec.class);
+    OutputStreamImpl mockOutputStream = mock(OutputStreamImpl.class);
+    when(mockInputOp.isKeyed()).thenReturn(true);
+    when(mockOutputStream.isKeyed()).thenReturn(true);
+    IntermediateMessageStreamImpl
+        mockPartitionedStream = new IntermediateMessageStreamImpl(mockGraph, mockInputOp, mockOutputStream);
+    when(mockGraph.getIntermediateStream(any(String.class), any(Serde.class))).thenReturn(mockPartitionedStream);
+
+    doAnswer(this.getRegisterMessageStreamAnswer()).when(mockContext).registerMessageStream(eq(3), any(MessageStream.class));
+    RexToJavaCompiler mockCompiler = mock(RexToJavaCompiler.class);
+    when(mockContext.getExpressionCompiler()).thenReturn(mockCompiler);
+    Expression mockExpr = mock(Expression.class);
+    when(mockCompiler.compile(any(), any())).thenReturn(mockExpr);
+
+    doAnswer(this.getRegisteredTableAnswer()).when(mockGraph).getTable(any(RocksDbTableDescriptor.class));
+    when(mockJoin.getJoinType()).thenReturn(JoinRelType.INNER);
+    SqlIOResolver mockResolver = mock(SqlIOResolver.class);
+    SqlIOConfig mockIOConfig = mock(SqlIOConfig.class);
+    TableDescriptor mockTableDesc = mock(TableDescriptor.class);
+    when(mockResolver.fetchSourceInfo(String.join(".", qualifiedTableName))).thenReturn(mockIOConfig);
+    when(mockIOConfig.getTableDescriptor()).thenReturn(Optional.of(mockTableDesc));
+
+    // Apply translate() method to verify that we are getting the correct map operator constructed
+    JoinTranslator joinTranslator = new JoinTranslator(3, mockResolver);
+    joinTranslator.translate(mockJoin, mockContext);
+    // make sure that context has been registered with LogicFilter and output message streams
+    verify(mockContext, times(1)).registerMessageStream(3, this.getRegisteredMessageStream(3));
+    when(mockContext.getRelNode(3)).thenReturn(mockJoin);
+    when(mockContext.getMessageStream(3)).thenReturn(this.getRegisteredMessageStream(3));
+    StreamTableJoinOperatorSpec
+        joinSpec = (StreamTableJoinOperatorSpec) Whitebox.getInternalState(this.getRegisteredMessageStream(3), "operatorSpec");
+    assertNotNull(joinSpec);
+    assertEquals(joinSpec.getOpCode(), OperatorSpec.OpCode.JOIN);
+
+    // Verify joinSpec has the corresponding setup
+    StreamTableJoinFunction joinFn = joinSpec.getJoinFn();
+    assertNotNull(joinFn);
+    assertTrue(Whitebox.getInternalState(joinFn, "isTablePosOnRight").equals(false));
+    assertEquals(new ArrayList<Integer>() {{ this.add(0); }}, Whitebox.getInternalState(joinFn, "streamFieldIds"));
+    assertEquals(leftFieldNames, Whitebox.getInternalState(joinFn, "tableFieldNames"));
+    List<String> outputFieldNames = new ArrayList<>();
+    outputFieldNames.addAll(leftFieldNames);
+    outputFieldNames.addAll(rightStreamFieldNames);
+    assertEquals(outputFieldNames, Whitebox.getInternalState(joinFn, "outFieldNames"));
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java
new file mode 100644
index 0000000..f84dd3f
--- /dev/null
+++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java
@@ -0,0 +1,289 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*   http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.sql.translator;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
+import org.apache.calcite.util.Pair;
+import org.apache.samza.config.Config;
+import org.apache.samza.container.TaskContextImpl;
+import org.apache.samza.container.TaskName;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.StreamGraphSpec;
+import org.apache.samza.operators.functions.MapFunction;
+import org.apache.samza.operators.functions.TimerFunction;
+import org.apache.samza.operators.functions.WatermarkFunction;
+import org.apache.samza.operators.spec.OperatorSpec;
+import org.apache.samza.operators.spec.StreamOperatorSpec;
+import org.apache.samza.sql.data.Expression;
+import org.apache.samza.sql.data.RexToJavaCompiler;
+import org.apache.samza.sql.data.SamzaSqlExecutionContext;
+import org.apache.samza.sql.data.SamzaSqlRelMessage;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.internal.util.reflection.Whitebox;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+
+/**
+ * Tests for {@link ProjectTranslator}
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest(LogicalProject.class)
+public class TestProjectTranslator extends TranslatorTestBase {
+  @Test
+  public void testTranslate() throws IOException, ClassNotFoundException {
+    // setup mock values to the constructor of FilterTranslator
+    LogicalProject mockProject = PowerMockito.mock(LogicalProject.class);
+    TranslatorContext mockContext = mock(TranslatorContext.class);
+    RelNode mockInput = mock(RelNode.class);
+    List<RelNode> inputs = new ArrayList<>();
+    inputs.add(mockInput);
+    when(mockInput.getId()).thenReturn(1);
+    when(mockProject.getId()).thenReturn(2);
+    when(mockProject.getInputs()).thenReturn(inputs);
+    when(mockProject.getInput()).thenReturn(mockInput);
+    RelDataType mockRowType = mock(RelDataType.class);
+    when(mockRowType.getFieldCount()).thenReturn(1);
+    when(mockProject.getRowType()).thenReturn(mockRowType);
+    RexNode mockRexField = mock(RexNode.class);
+    List<Pair<RexNode, String>> namedProjects = new ArrayList<>();
+    namedProjects.add(Pair.of(mockRexField, "test_field"));
+    when(mockProject.getNamedProjects()).thenReturn(namedProjects);
+    StreamGraphSpec mockGraph = mock(StreamGraphSpec.class);
+    OperatorSpec<Object, SamzaSqlRelMessage> mockInputOp = mock(OperatorSpec.class);
+    MessageStream<SamzaSqlRelMessage> mockStream = new MessageStreamImpl<>(mockGraph, mockInputOp);
+    when(mockContext.getMessageStream(eq(1))).thenReturn(mockStream);
+    doAnswer(this.getRegisterMessageStreamAnswer()).when(mockContext).registerMessageStream(eq(2), any(MessageStream.class));
+    RexToJavaCompiler mockCompiler = mock(RexToJavaCompiler.class);
+    when(mockContext.getExpressionCompiler()).thenReturn(mockCompiler);
+    Expression mockExpr = mock(Expression.class);
+    when(mockCompiler.compile(any(), any())).thenReturn(mockExpr);
+
+    // Apply translate() method to verify that we are getting the correct map operator constructed
+    ProjectTranslator projectTranslator = new ProjectTranslator();
+    projectTranslator.translate(mockProject, mockContext);
+    // make sure that context has been registered with LogicFilter and output message streams
+    verify(mockContext, times(1)).registerRelNode(2, mockProject);
+    verify(mockContext, times(1)).registerMessageStream(2, this.getRegisteredMessageStream(2));
+    when(mockContext.getRelNode(2)).thenReturn(mockProject);
+    when(mockContext.getMessageStream(2)).thenReturn(this.getRegisteredMessageStream(2));
+    StreamOperatorSpec projectSpec = (StreamOperatorSpec) Whitebox.getInternalState(this.getRegisteredMessageStream(2), "operatorSpec");
+    assertNotNull(projectSpec);
+    assertEquals(projectSpec.getOpCode(), OperatorSpec.OpCode.MAP);
+
+    // Verify that the init() method will establish the context for the map function
+    Config mockConfig = mock(Config.class);
+    TaskContextImpl taskContext = new TaskContextImpl(new TaskName("Partition-1"), null, null,
+        new HashSet<>(), null, null, null, null, null, null);
+    taskContext.setUserContext(mockContext);
+    projectSpec.getTransformFn().init(mockConfig, taskContext);
+    MapFunction mapFn = (MapFunction) Whitebox.getInternalState(projectSpec, "mapFn");
+    assertNotNull(mapFn);
+    assertEquals(mockContext, Whitebox.getInternalState(mapFn, "context"));
+    assertEquals(mockProject, Whitebox.getInternalState(mapFn, "project"));
+    assertEquals(mockExpr, Whitebox.getInternalState(mapFn, "expr"));
+
+    // Calling mapFn.apply() to verify the filter function is correctly applied to the input message
+    SamzaSqlRelMessage mockInputMsg = new SamzaSqlRelMessage(new ArrayList<>(), new ArrayList<>());
+    SamzaSqlExecutionContext executionContext = mock(SamzaSqlExecutionContext.class);
+    DataContext dataContext = mock(DataContext.class);
+    when(mockContext.getExecutionContext()).thenReturn(executionContext);
+    when(mockContext.getDataContext()).thenReturn(dataContext);
+    Object[] result = new Object[1];
+    final Object mockFieldObj = new Object();
+
+    doAnswer( invocation -> {
+      Object[] retValue = invocation.getArgumentAt(3, Object[].class);
+      retValue[0] = mockFieldObj;
+      return null;
+    }).when(mockExpr).execute(eq(executionContext), eq(dataContext),
+        eq(mockInputMsg.getSamzaSqlRelRecord().getFieldValues().toArray()), eq(result));
+    SamzaSqlRelMessage retMsg = (SamzaSqlRelMessage) mapFn.apply(mockInputMsg);
+    assertEquals(retMsg.getSamzaSqlRelRecord().getFieldNames(),
+        new ArrayList<String>() {{
+          this.add("test_field");
+        }});
+    assertEquals(retMsg.getSamzaSqlRelRecord().getFieldValues(), new ArrayList<Object>() {{
+          this.add(mockFieldObj);
+        }});
+
+  }
+
+  @Test
+  public void testTranslateWithFlatten() throws IOException, ClassNotFoundException {
+    // setup mock values to the constructor of ProjectTranslator
+    LogicalProject mockProject = PowerMockito.mock(LogicalProject.class);
+    TranslatorContext mockContext = mock(TranslatorContext.class);
+    RelNode mockInput = mock(RelNode.class);
+    List<RelNode> inputs = new ArrayList<>();
+    inputs.add(mockInput);
+    when(mockInput.getId()).thenReturn(1);
+    when(mockProject.getId()).thenReturn(2);
+    when(mockProject.getInputs()).thenReturn(inputs);
+    when(mockProject.getInput()).thenReturn(mockInput);
+    RelDataType mockRowType = mock(RelDataType.class);
+    when(mockRowType.getFieldCount()).thenReturn(1);
+    when(mockProject.getRowType()).thenReturn(mockRowType);
+    RexNode mockRexField = mock(RexNode.class);
+    List<Pair<RexNode, String>> namedProjects = new ArrayList<>();
+    namedProjects.add(Pair.of(mockRexField, "test_field"));
+    when(mockProject.getNamedProjects()).thenReturn(namedProjects);
+    List<RexNode> flattenProjects = new ArrayList<>();
+    RexCall mockFlattenProject = mock(RexCall.class);
+    SqlUserDefinedFunction sqlFlattenUdf = mock(SqlUserDefinedFunction.class);
+    when(sqlFlattenUdf.getName()).thenReturn("flatten");
+    List<RexNode> flattenUdfOperands = new ArrayList<>();
+    RexInputRef rexInputRef = mock(RexInputRef.class);
+    when(rexInputRef.getIndex()).thenReturn(0);
+    flattenUdfOperands.add(rexInputRef);
+    when(mockFlattenProject.getOperands()).thenReturn(flattenUdfOperands);
+    Whitebox.setInternalState(mockFlattenProject, "op", sqlFlattenUdf);
+    flattenProjects.add(mockFlattenProject);
+    when(mockProject.getProjects()).thenReturn(flattenProjects);
+
+    StreamGraphSpec mockGraph = mock(StreamGraphSpec.class);
+    OperatorSpec<Object, SamzaSqlRelMessage> mockInputOp = new OperatorSpec(OperatorSpec.OpCode.INPUT, "1") {
+
+      @Override
+      public WatermarkFunction getWatermarkFn() {
+        return null;
+      }
+
+      @Override
+      public TimerFunction getTimerFn() {
+        return null;
+      }
+    };
+
+    MessageStream<SamzaSqlRelMessage> mockStream = new MessageStreamImpl<>(mockGraph, mockInputOp);
+    when(mockContext.getMessageStream(eq(1))).thenReturn(mockStream);
+    doAnswer(this.getRegisterMessageStreamAnswer()).when(mockContext).registerMessageStream(eq(2), any(MessageStream.class));
+    RexToJavaCompiler mockCompiler = mock(RexToJavaCompiler.class);
+    when(mockContext.getExpressionCompiler()).thenReturn(mockCompiler);
+    Expression mockExpr = mock(Expression.class);
+    when(mockCompiler.compile(any(), any())).thenReturn(mockExpr);
+
+    // Apply translate() method to verify that we are getting the correct map operator constructed
+    ProjectTranslator projectTranslator = new ProjectTranslator();
+    projectTranslator.translate(mockProject, mockContext);
+    // make sure that context has been registered with LogicFilter and output message streams
+    verify(mockContext, times(1)).registerRelNode(2, mockProject);
+    verify(mockContext, times(1)).registerMessageStream(2, this.getRegisteredMessageStream(2));
+    when(mockContext.getRelNode(2)).thenReturn(mockProject);
+    when(mockContext.getMessageStream(2)).thenReturn(this.getRegisteredMessageStream(2));
+
+
+    Collection<OperatorSpec>
+        nextOps = ((OperatorSpec) Whitebox.getInternalState(mockStream, "operatorSpec")).getRegisteredOperatorSpecs();
+    StreamOperatorSpec flattenOp = (StreamOperatorSpec) nextOps.iterator().next();
+    assertNotNull(flattenOp);
+    Object testObj = new Object();
+    SamzaSqlRelMessage mockMsg = new SamzaSqlRelMessage(new ArrayList<String>() {{
+      this.add("test_field_no1");
+    }}, new ArrayList<Object>() {{
+      this.add(testObj);
+    }});
+    Collection<SamzaSqlRelMessage> flattenedMsgs = flattenOp.getTransformFn().apply(mockMsg);
+    assertTrue(flattenedMsgs.size() == 1);
+    assertTrue(flattenedMsgs.stream().anyMatch(s -> s.getSamzaSqlRelRecord().getFieldValues().get(0).equals(testObj)));
+    List<Integer> testList = new ArrayList<>();
+    for (int i = 0; i < 10; i++) {
+      testList.add(new Integer(i));
+    }
+    mockMsg = new SamzaSqlRelMessage(new ArrayList<String>() {{
+      this.add("test_list_field1");
+    }}, new ArrayList<Object>() {{
+      this.add(testList);
+    }});
+    flattenedMsgs = flattenOp.getTransformFn().apply(mockMsg);
+    assertTrue(flattenedMsgs.size() == 10);
+    List<Integer> actualList = flattenedMsgs.stream()
+        .map(m -> ((List<Integer>) m.getSamzaSqlRelRecord().getFieldValues().get(0)).get(0))
+        .collect(ArrayList::new, (c, a) -> c.add(a), (c1, c2) -> c1.addAll(c2));
+    assertEquals(testList, actualList);
+
+    StreamOperatorSpec projectSpec = (StreamOperatorSpec) Whitebox.getInternalState(this.getRegisteredMessageStream(2), "operatorSpec");
+    assertNotNull(projectSpec);
+    assertEquals(projectSpec.getOpCode(), OperatorSpec.OpCode.MAP);
+
+    // Verify that the init() method will establish the context for the map function
+    Config mockConfig = mock(Config.class);
+    TaskContextImpl taskContext = new TaskContextImpl(new TaskName("Partition-1"), null, null,
+        new HashSet<>(), null, null, null, null, null, null);
+    taskContext.setUserContext(mockContext);
+    projectSpec.getTransformFn().init(mockConfig, taskContext);
+    MapFunction mapFn = (MapFunction) Whitebox.getInternalState(projectSpec, "mapFn");
+    assertNotNull(mapFn);
+    assertEquals(mockContext, Whitebox.getInternalState(mapFn, "context"));
+    assertEquals(mockProject, Whitebox.getInternalState(mapFn, "project"));
+    assertEquals(mockExpr, Whitebox.getInternalState(mapFn, "expr"));
+
+    // Calling mapFn.apply() to verify the filter function is correctly applied to the input message
+    SamzaSqlRelMessage mockInputMsg = new SamzaSqlRelMessage(new ArrayList<>(), new ArrayList<>());
+    SamzaSqlExecutionContext executionContext = mock(SamzaSqlExecutionContext.class);
+    DataContext dataContext = mock(DataContext.class);
+    when(mockContext.getExecutionContext()).thenReturn(executionContext);
+    when(mockContext.getDataContext()).thenReturn(dataContext);
+    Object[] result = new Object[1];
+    final Object mockFieldObj = new Object();
+
+    doAnswer( invocation -> {
+      Object[] retValue = invocation.getArgumentAt(3, Object[].class);
+      retValue[0] = mockFieldObj;
+      return null;
+    }).when(mockExpr).execute(eq(executionContext), eq(dataContext),
+        eq(mockInputMsg.getSamzaSqlRelRecord().getFieldValues().toArray()), eq(result));
+    SamzaSqlRelMessage retMsg = (SamzaSqlRelMessage) mapFn.apply(mockInputMsg);
+    assertEquals(retMsg.getSamzaSqlRelRecord().getFieldNames(),
+        new ArrayList<String>() {{
+          this.add("test_field");
+        }});
+    assertEquals(retMsg.getSamzaSqlRelRecord().getFieldValues(), new ArrayList<Object>() {{
+      this.add(mockFieldObj);
+    }});
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java
new file mode 100644
index 0000000..65b8c8c
--- /dev/null
+++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java
@@ -0,0 +1,596 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*   http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.sql.translator;
+
+import java.util.HashSet;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.container.TaskContextImpl;
+import org.apache.samza.container.TaskName;
+import org.apache.samza.operators.OperatorSpecGraph;
+import org.apache.samza.operators.StreamGraphSpec;
+import org.apache.samza.runtime.LocalApplicationRunner;
+import org.apache.samza.sql.data.SamzaSqlExecutionContext;
+import org.apache.samza.operators.spec.OperatorSpec;
+import org.apache.samza.sql.impl.ConfigBasedIOResolverFactory;
+import org.apache.samza.sql.runner.SamzaSqlApplicationConfig;
+import org.apache.samza.sql.runner.SamzaSqlApplicationRunner;
+import org.apache.samza.sql.testutil.SamzaSqlQueryParser;
+import org.apache.samza.sql.testutil.SamzaSqlTestConfig;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
+
+public class TestQueryTranslator {
+
+  // Helper functions to validate the cloned copies of TranslatorContext and SamzaSqlExecutionContext
+  private void validateClonedTranslatorContext(TranslatorContext originContext, TranslatorContext clonedContext) {
+    Assert.assertNotEquals(originContext, clonedContext);
+    Assert.assertTrue(originContext.getExpressionCompiler() == clonedContext.getExpressionCompiler());
+    Assert.assertTrue(originContext.getStreamGraph() == clonedContext.getStreamGraph());
+    Assert.assertTrue(originContext.getExpressionCompiler() == clonedContext.getExpressionCompiler());
+    Assert.assertTrue(Whitebox.getInternalState(originContext, "relSamzaConverters") == Whitebox.getInternalState(clonedContext, "relSamzaConverters"));
+    Assert.assertTrue(Whitebox.getInternalState(originContext, "messsageStreams") == Whitebox.getInternalState(clonedContext, "messsageStreams"));
+    Assert.assertTrue(Whitebox.getInternalState(originContext, "relNodes") == Whitebox.getInternalState(clonedContext, "relNodes"));
+    Assert.assertNotEquals(originContext.getDataContext(), clonedContext.getDataContext());
+    validateClonedExecutionContext(originContext.getExecutionContext(), clonedContext.getExecutionContext());
+  }
+
+  private void validateClonedExecutionContext(SamzaSqlExecutionContext originContext,
+      SamzaSqlExecutionContext clonedContext) {
+    Assert.assertNotEquals(originContext, clonedContext);
+    Assert.assertTrue(
+        Whitebox.getInternalState(originContext, "sqlConfig") == Whitebox.getInternalState(clonedContext, "sqlConfig"));
+    Assert.assertTrue(Whitebox.getInternalState(originContext, "udfMetadata") == Whitebox.getInternalState(clonedContext,
+        "udfMetadata"));
+    Assert.assertTrue(Whitebox.getInternalState(originContext, "udfInstances") != Whitebox.getInternalState(clonedContext,
+        "udfInstances"));
+  }
+
+  private final Map<String, String> configs = new HashMap<>();
+
+  @Before
+  public void setUp() {
+    configs.put("job.default.system", "kafka");
+  }
+
+  @Test
+  public void testTranslate() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10);
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT,
+        "Insert into testavro.outputTopic select MyTest(id) from testavro.level1.level2.SIMPLE1 as s where s.id = 10");
+    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
+    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
+    StreamGraphSpec
+        graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig);
+    translator.translate(queryInfo, graphSpec);
+    OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph();
+    Assert.assertEquals(1, specGraph.getOutputStreams().size());
+    Assert.assertEquals("testavro", specGraph.getOutputStreams().keySet().stream().findFirst().get().getSystemName());
+    Assert.assertEquals("outputTopic", specGraph.getOutputStreams().keySet().stream().findFirst().get().getPhysicalName());
+    Assert.assertEquals(1, specGraph.getInputOperators().size());
+    Assert.assertEquals("testavro",
+        specGraph.getInputOperators().keySet().stream().findFirst().get().getSystemName());
+    Assert.assertEquals("SIMPLE1",
+        specGraph.getInputOperators().keySet().stream().findFirst().get().getPhysicalName());
+
+    validatePerTaskContextInit(graphSpec, samzaConfig);
+  }
+
+  private void validatePerTaskContextInit(StreamGraphSpec graphSpec, Config samzaConfig) {
+    // make sure that each task context would have a separate instance of cloned TranslatorContext
+    TaskContextImpl testContext = new TaskContextImpl(new TaskName("Partition 1"), null, null,
+        new HashSet<>(), null, null, null, null, null, null);
+    // call ContextManager.init() to instantiate the per-task TranslatorContext
+    graphSpec.getContextManager().init(samzaConfig, testContext);
+    Assert.assertNotNull(testContext.getUserContext());
+    Assert.assertTrue(testContext.getUserContext() instanceof TranslatorContext);
+    TranslatorContext contextPerTaskOne = (TranslatorContext) testContext.getUserContext();
+    // call ContextManager.init() second time to instantiate another clone of TranslatorContext
+    graphSpec.getContextManager().init(samzaConfig, testContext);
+    Assert.assertTrue(testContext.getUserContext() instanceof TranslatorContext);
+    // validate the two copies of TranslatorContext are clones of each other
+    validateClonedTranslatorContext(contextPerTaskOne, (TranslatorContext) testContext.getUserContext());
+  }
+
+  @Test
+  public void testTranslateComplex() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10);
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT,
+        "Insert into testavro.outputTopic select Flatten(array_values) from testavro.COMPLEX1");
+//    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT,
+//        "Insert into testavro.foo2 select string_value, SUM(id) from testavro.COMPLEX1 "
+//            + "GROUP BY TumbleWindow(CURRENT_TIME, INTERVAL '1' HOUR), string_value");
+    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
+    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
+    StreamGraphSpec
+        graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig);
+    translator.translate(queryInfo, graphSpec);
+    OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph();
+    Assert.assertEquals(1, specGraph.getOutputStreams().size());
+    Assert.assertEquals("testavro", specGraph.getOutputStreams().keySet().stream().findFirst().get().getSystemName());
+    Assert.assertEquals("outputTopic", specGraph.getOutputStreams().keySet().stream().findFirst().get().getPhysicalName());
+    Assert.assertEquals(1, specGraph.getInputOperators().size());
+    Assert.assertEquals("testavro",
+        specGraph.getInputOperators().keySet().stream().findFirst().get().getSystemName());
+    Assert.assertEquals("COMPLEX1",
+        specGraph.getInputOperators().keySet().stream().findFirst().get().getPhysicalName());
+
+    validatePerTaskContextInit(graphSpec, samzaConfig);
+  }
+
+  @Test
+  public void testTranslateSubQuery() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10);
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT,
+        "Insert into testavro.outputTopic select Flatten(a), id from (select id, array_values a, string_value s from testavro.COMPLEX1)");
+    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
+    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
+    StreamGraphSpec
+        graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig);
+    translator.translate(queryInfo, graphSpec);
+    OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph();
+    Assert.assertEquals(1, specGraph.getOutputStreams().size());
+    Assert.assertEquals("testavro", specGraph.getOutputStreams().keySet().stream().findFirst().get().getSystemName());
+    Assert.assertEquals("outputTopic", specGraph.getOutputStreams().keySet().stream().findFirst().get().getPhysicalName());
+    Assert.assertEquals(1, specGraph.getInputOperators().size());
+    Assert.assertEquals("testavro",
+        specGraph.getInputOperators().keySet().stream().findFirst().get().getSystemName());
+    Assert.assertEquals("COMPLEX1",
+        specGraph.getInputOperators().keySet().stream().findFirst().get().getPhysicalName());
+
+    validatePerTaskContextInit(graphSpec, samzaConfig);
+  }
+
+  @Test (expected = SamzaException.class)
+  public void testTranslateStreamTableJoinWithoutJoinOperator() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
+    String sql =
+        "Insert into testavro.enrichedPageViewTopic"
+            + " select p.name as profileName, pv.pageKey"
+            + " from testavro.PAGEVIEW as pv, testavro.PROFILE.`$table` as p"
+            + " where p.id = pv.profileId";
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
+    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
+    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
+    StreamGraphSpec
+        graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig);
+    translator.translate(queryInfo, graphSpec);
+  }
+
+  @Test (expected = SamzaException.class)
+  public void testTranslateStreamTableJoinWithFullJoinOperator() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
+    String sql =
+        "Insert into testavro.enrichedPageViewTopic"
+            + " select p.name as profileName, pv.pageKey"
+            + " from testavro.PAGEVIEW as pv"
+            + " full join testavro.PROFILE.`$table` as p"
+            + " on p.id = pv.profileId";
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
+    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
+    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
+    StreamGraphSpec
+        graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig);
+    translator.translate(queryInfo, graphSpec);
+  }
+
+  @Test (expected = IllegalStateException.class)
+  public void testTranslateStreamTableJoinWithSelfJoinOperator() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
+    String sql =
+        "Insert into testavro.enrichedPageViewTopic"
+            + " select p1.name as profileName"
+            + " from testavro.PROFILE.`$table` as p1"
+            + " join testavro.PROFILE.`$table` as p2"
+            + " on p1.id = p2.id";
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
+    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
+    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
+    StreamGraphSpec
+        graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig);
+    translator.translate(queryInfo, graphSpec);
+  }
+
+  @Test (expected = SamzaException.class)
+  public void testTranslateStreamTableJoinWithThetaCondition() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
+    String sql =
+        "Insert into testavro.enrichedPageViewTopic"
+            + " select p.name as profileName, pv.pageKey"
+            + " from testavro.PAGEVIEW as pv"
+            + " join testavro.PROFILE.`$table` as p"
+            + " on p.id <> pv.profileId";
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
+    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
+    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
+    StreamGraphSpec
+        graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig);
+    translator.translate(queryInfo, graphSpec);
+  }
+
+  @Test (expected = SamzaException.class)
+  public void testTranslateStreamTableCrossJoin() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
+    String sql =
+        "Insert into testavro.enrichedPageViewTopic"
+            + " select p.name as profileName, pv.pageKey"
+            + " from testavro.PAGEVIEW as pv, testavro.PROFILE.`$table` as p";
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
+    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
+    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
+    StreamGraphSpec
+        graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig);
+    translator.translate(queryInfo, graphSpec);
+  }
+
+  @Test (expected = SamzaException.class)
+  public void testTranslateStreamTableJoinWithAndLiteralCondition() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
+    String sql =
+        "Insert into testavro.enrichedPageViewTopic"
+            + " select p.name as profileName, pv.pageKey"
+            + " from testavro.PAGEVIEW as pv"
+            + " join testavro.PROFILE.`$table` as p"
+            + " on p.id = pv.profileId and p.name = 'John'";
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
+    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
+    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
+    StreamGraphSpec
+        graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig);
+    translator.translate(queryInfo, graphSpec);
+  }
+
+  @Test (expected = SamzaException.class)
+  public void testTranslateStreamTableJoinWithSubQuery() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
+    String sql =
+        "Insert into testavro.enrichedPageViewTopic"
+            + " select p.name as profileName, pv.pageKey"
+            + " from testavro.PAGEVIEW as pv"
+            + " where exists "
+            + " (select p.id from testavro.PROFILE.`$table` as p"
+            + " where p.id = pv.profileId)";
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
+    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
+    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
+    StreamGraphSpec
+        graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig);
+    translator.translate(queryInfo, graphSpec);
+  }
+
+  @Test (expected = SamzaException.class)
+  public void testTranslateTableTableJoin() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
+    String sql =
+        "Insert into testavro.enrichedPageViewTopic"
+            + " select p.name as profileName, pv.pageKey"
+            + " from testavro.PAGEVIEW.`$table` as pv"
+            + " join testavro.PROFILE.`$table` as p"
+            + " on p.id = pv.profileId";
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
+    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
+    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
+    StreamGraphSpec
+        graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig);
+    translator.translate(queryInfo, graphSpec);
+  }
+
+  @Test (expected = SamzaException.class)
+  public void testTranslateStreamStreamJoin() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
+    String sql =
+        "Insert into testavro.enrichedPageViewTopic"
+            + " select p.name as profileName, pv.pageKey"
+            + " from testavro.PAGEVIEW as pv"
+            + " join testavro.PROFILE as p"
+            + " on p.id = pv.profileId";
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
+    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
+    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
+    StreamGraphSpec
+        graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig);
+    translator.translate(queryInfo, graphSpec);
+  }
+
+  @Test (expected = SamzaException.class)
+  public void testTranslateJoinWithIncorrectLeftJoin() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
+    String sql =
+        "Insert into testavro.enrichedPageViewTopic"
+            + " select p.name as profileName, pv.pageKey"
+            + " from testavro.PAGEVIEW.`$table` as pv"
+            + " left join testavro.PROFILE as p"
+            + " on p.id = pv.profileId";
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
+    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
+    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
+    StreamGraphSpec
+        graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig);
+    translator.translate(queryInfo, graphSpec);
+  }
+
+  @Test (expected = SamzaException.class)
+  public void testTranslateJoinWithIncorrectRightJoin() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
+    String sql =
+        "Insert into testavro.enrichedPageViewTopic"
+            + " select p.name as profileName, pv.pageKey"
+            + " from testavro.PAGEVIEW as pv"
+            + " right join testavro.PROFILE.`$table` as p"
+            + " on p.id = pv.profileId";
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
+    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
+    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
+    StreamGraphSpec
+        graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig);
+    translator.translate(queryInfo, graphSpec);
+  }
+
+  @Test (expected = SamzaException.class)
+  public void testTranslateStreamTableInnerJoinWithMissingStream() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 10);
+    String configIOResolverDomain =
+        String.format(SamzaSqlApplicationConfig.CFG_FMT_SOURCE_RESOLVER_DOMAIN, "config");
+    config.put(configIOResolverDomain + SamzaSqlApplicationConfig.CFG_FACTORY,
+        ConfigBasedIOResolverFactory.class.getName());
+    String sql =
+        "Insert into testavro.enrichedPageViewTopic"
+            + " select p.name as profileName, pv.pageKey"
+            + " from testavro.PAGEVIEW as pv"
+            + " join testavro.`$table` as p"
+            + " on p.id = pv.profileId";
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
+    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
+    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
+    StreamGraphSpec
+        graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig);
+    translator.translate(queryInfo, graphSpec);
+  }
+
+  @Test (expected = SamzaException.class)
+  public void testTranslateStreamTableInnerJoinWithUdf() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 10);
+    String sql =
+        "Insert into testavro.enrichedPageViewTopic"
+            + " select p.name as profileName, pv.pageKey"
+            + " from testavro.PAGEVIEW as pv"
+            + " join testavro.PROFILE.`$table` as p"
+            + " on MyTest(p.id) = MyTest(pv.profileId)";
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
+    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
+    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
+    StreamGraphSpec
+        graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig);
+    translator.translate(queryInfo, graphSpec);
+  }
+
+  @Test
+  public void testTranslateStreamTableInnerJoin() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 10);
+    String sql =
+        "Insert into testavro.enrichedPageViewTopic"
+            + " select p.name as profileName, pv.pageKey"
+            + " from testavro.PAGEVIEW as pv"
+            + " join testavro.PROFILE.`$table` as p"
+            + " on p.id = pv.profileId";
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
+    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
+    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
+    StreamGraphSpec
+        graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig);
+    translator.translate(queryInfo, graphSpec);
+    OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph();
+
+    Assert.assertEquals(2, specGraph.getOutputStreams().size());
+    Assert.assertEquals("kafka", specGraph.getOutputStreams().keySet().stream().findFirst().get().getSystemName());
+    Assert.assertEquals("sql-job-1-partition_by-stream_1", specGraph.getOutputStreams().keySet().stream().findFirst().get().getPhysicalName());
+    Assert.assertEquals("testavro", specGraph.getOutputStreams().keySet().stream().skip(1).findFirst().get().getSystemName());
+    Assert.assertEquals("enrichedPageViewTopic", specGraph.getOutputStreams().keySet().stream().skip(1).findFirst().get().getPhysicalName());
+
+    Assert.assertEquals(3, specGraph.getInputOperators().size());
+    Assert.assertEquals("testavro",
+        specGraph.getInputOperators().keySet().stream().findFirst().get().getSystemName());
+    Assert.assertEquals("PAGEVIEW",
+        specGraph.getInputOperators().keySet().stream().findFirst().get().getPhysicalName());
+    Assert.assertEquals("testavro",
+        specGraph.getInputOperators().keySet().stream().skip(1).findFirst().get().getSystemName());
+    Assert.assertEquals("PROFILE",
+        specGraph.getInputOperators().keySet().stream().skip(1).findFirst().get().getPhysicalName());
+    Assert.assertEquals("kafka",
+        specGraph.getInputOperators().keySet().stream().skip(2).findFirst().get().getSystemName());
+    Assert.assertEquals("sql-job-1-partition_by-stream_1",
+        specGraph.getInputOperators().keySet().stream().skip(2).findFirst().get().getPhysicalName());
+
+    validatePerTaskContextInit(graphSpec, samzaConfig);
+  }
+
+  @Test
+  public void testTranslateStreamTableLeftJoin() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 10);
+    String sql =
+        "Insert into testavro.enrichedPageViewTopic"
+            + " select p.name as profileName, pv.pageKey"
+            + " from testavro.PAGEVIEW as pv"
+            + " left join testavro.PROFILE.`$table` as p"
+            + " on p.id = pv.profileId";
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
+    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
+    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
+    StreamGraphSpec
+        graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig);
+    translator.translate(queryInfo, graphSpec);
+
+    OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph();
+
+    Assert.assertEquals(2, specGraph.getOutputStreams().size());
+    Assert.assertEquals("kafka", specGraph.getOutputStreams().keySet().stream().findFirst().get().getSystemName());
+    Assert.assertEquals("sql-job-1-partition_by-stream_1",
+        specGraph.getOutputStreams().keySet().stream().findFirst().get().getPhysicalName());
+    Assert.assertEquals("testavro", specGraph.getOutputStreams().keySet().stream().skip(1).findFirst().get().getSystemName());
+    Assert.assertEquals("enrichedPageViewTopic",
+        specGraph.getOutputStreams().keySet().stream().skip(1).findFirst().get().getPhysicalName());
+
+    Assert.assertEquals(3, specGraph.getInputOperators().size());
+    Assert.assertEquals("testavro",
+        specGraph.getInputOperators().keySet().stream().findFirst().get().getSystemName());
+    Assert.assertEquals("PAGEVIEW",
+        specGraph.getInputOperators().keySet().stream().findFirst().get().getPhysicalName());
+    Assert.assertEquals("testavro",
+        specGraph.getInputOperators().keySet().stream().skip(1).findFirst().get().getSystemName());
+    Assert.assertEquals("PROFILE",
+        specGraph.getInputOperators().keySet().stream().skip(1).findFirst().get().getPhysicalName());
+    Assert.assertEquals("kafka",
+        specGraph.getInputOperators().keySet().stream().skip(2).findFirst().get().getSystemName());
+    Assert.assertEquals("sql-job-1-partition_by-stream_1",
+        specGraph.getInputOperators().keySet().stream().skip(2).findFirst().get().getPhysicalName());
+
+    validatePerTaskContextInit(graphSpec, samzaConfig);
+  }
+
+  @Test
+  public void testTranslateStreamTableRightJoin() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 10);
+    String sql =
+        "Insert into testavro.enrichedPageViewTopic"
+            + " select p.name as profileName, pv.pageKey"
+            + " from testavro.PROFILE.`$table` as p"
+            + " right join testavro.PAGEVIEW as pv"
+            + " on p.id = pv.profileId";
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
+    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
+    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
+    StreamGraphSpec
+        graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig);
+    translator.translate(queryInfo, graphSpec);
+
+    OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph();
+
+    Assert.assertEquals(2, specGraph.getOutputStreams().size());
+    Assert.assertEquals("kafka", specGraph.getOutputStreams().keySet().stream().findFirst().get().getSystemName());
+    Assert.assertEquals("sql-job-1-partition_by-stream_1",
+        specGraph.getOutputStreams().keySet().stream().findFirst().get().getPhysicalName());
+    Assert.assertEquals("testavro", specGraph.getOutputStreams().keySet().stream().skip(1).findFirst().get().getSystemName());
+    Assert.assertEquals("enrichedPageViewTopic",
+        specGraph.getOutputStreams().keySet().stream().skip(1).findFirst().get().getPhysicalName());
+
+    Assert.assertEquals(3, specGraph.getInputOperators().size());
+    Assert.assertEquals("testavro",
+        specGraph.getInputOperators().keySet().stream().findFirst().get().getSystemName());
+    Assert.assertEquals("PROFILE",
+        specGraph.getInputOperators().keySet().stream().findFirst().get().getPhysicalName());
+    Assert.assertEquals("testavro",
+        specGraph.getInputOperators().keySet().stream().skip(1).findFirst().get().getSystemName());
+    Assert.assertEquals("PAGEVIEW",
+        specGraph.getInputOperators().keySet().stream().skip(1).findFirst().get().getPhysicalName());
+    Assert.assertEquals("kafka",
+        specGraph.getInputOperators().keySet().stream().skip(2).findFirst().get().getSystemName());
+    Assert.assertEquals("sql-job-1-partition_by-stream_1",
+        specGraph.getInputOperators().keySet().stream().skip(2).findFirst().get().getPhysicalName());
+
+    validatePerTaskContextInit(graphSpec, samzaConfig);
+  }
+
+  @Test
+  public void testTranslateGroupBy() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 10);
+    String sql =
+        "Insert into testavro.pageViewCountTopic"
+            + " select 'SampleJob' as jobName, pv.pageKey, count(*) as `count`"
+            + " from testavro.PAGEVIEW as pv"
+            + " where pv.pageKey = 'job' or pv.pageKey = 'inbox'"
+            + " group by (pv.pageKey)";
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
+    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
+    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
+    StreamGraphSpec
+        graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig);
+    translator.translate(queryInfo, graphSpec);
+    OperatorSpecGraph specGraph = graphSpec.getOperatorSpecGraph();
+
+    Assert.assertEquals(1, specGraph.getInputOperators().size());
+    Assert.assertEquals(1, specGraph.getOutputStreams().size());
+    Assert.assertTrue(specGraph.hasWindowOrJoins());
+    Collection<OperatorSpec> operatorSpecs = specGraph.getAllOperatorSpecs();
+  }
+
+  @Test (expected = SamzaException.class)
+  public void testTranslateGroupByWithSumAggregator() {
+    Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 10);
+    String sql =
+        "Insert into testavro.pageViewCountTopic"
+            + " select 'SampleJob' as jobName, pv.pageKey, sum(pv.profileId) as `sum`"
+            + " from testavro.PAGEVIEW as pv" + " where pv.pageKey = 'job' or pv.pageKey = 'inbox'"
+            + " group by (pv.pageKey)";
+    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
+    Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+    QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
+    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
+    StreamGraphSpec
+        graphSpec = new StreamGraphSpec(new LocalApplicationRunner(samzaConfig), samzaConfig);
+    translator.translate(queryInfo, graphSpec);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/test/java/org/apache/samza/sql/translator/TestSamzaSqlRelMessageJoinFunction.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestSamzaSqlRelMessageJoinFunction.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestSamzaSqlRelMessageJoinFunction.java
new file mode 100644
index 0000000..5dd2d21
--- /dev/null
+++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestSamzaSqlRelMessageJoinFunction.java
@@ -0,0 +1,118 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*   http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.sql.translator;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.samza.operators.KV;
+import org.apache.samza.sql.data.SamzaSqlCompositeKey;
+import org.apache.samza.sql.data.SamzaSqlRelMessage;
+import org.junit.Assert;
+import org.junit.Test;
+
+
+public class TestSamzaSqlRelMessageJoinFunction {
+
+  private List<String> streamFieldNames = Arrays.asList("field1", "field2", "field3", "field4");
+  private List<Object> streamFieldValues = Arrays.asList("value1", 1, null, "value4");
+  private List<String> tableFieldNames = Arrays.asList("field11", "field12", "field13", "field14");
+  private List<Object> tableFieldValues = Arrays.asList("value1", 1, null, "value5");
+
+  @Test
+  public void testWithInnerJoinWithTableOnRight() {
+    SamzaSqlRelMessage streamMsg = new SamzaSqlRelMessage(streamFieldNames, streamFieldValues);
+    SamzaSqlRelMessage tableMsg = new SamzaSqlRelMessage(tableFieldNames, tableFieldValues);
+    JoinRelType joinRelType = JoinRelType.INNER;
+    List<Integer> streamKeyIds = Arrays.asList(0, 1);
+    List<Integer> tableKeyIds = Arrays.asList(0, 1);
+    SamzaSqlCompositeKey compositeKey = SamzaSqlCompositeKey.createSamzaSqlCompositeKey(tableMsg, tableKeyIds);
+    KV<SamzaSqlCompositeKey, SamzaSqlRelMessage> record = KV.of(compositeKey, tableMsg);
+
+    SamzaSqlRelMessageJoinFunction joinFn =
+        new SamzaSqlRelMessageJoinFunction(joinRelType, true, streamKeyIds, streamFieldNames, tableFieldNames);
+    SamzaSqlRelMessage outMsg = joinFn.apply(streamMsg, record);
+
+    Assert.assertEquals(outMsg.getSamzaSqlRelRecord().getFieldValues().size(),
+        outMsg.getSamzaSqlRelRecord().getFieldNames().size());
+    List<String> expectedFieldNames = new ArrayList<>(streamFieldNames);
+    expectedFieldNames.addAll(tableFieldNames);
+    List<Object> expectedFieldValues = new ArrayList<>(streamFieldValues);
+    expectedFieldValues.addAll(tableFieldValues);
+    Assert.assertEquals(outMsg.getSamzaSqlRelRecord().getFieldValues(), expectedFieldValues);
+  }
+
+  @Test
+  public void testWithInnerJoinWithTableOnLeft() {
+    SamzaSqlRelMessage streamMsg = new SamzaSqlRelMessage(streamFieldNames, streamFieldValues);
+    SamzaSqlRelMessage tableMsg = new SamzaSqlRelMessage(tableFieldNames, tableFieldValues);
+    JoinRelType joinRelType = JoinRelType.INNER;
+    List<Integer> streamKeyIds = Arrays.asList(0, 2);
+    List<Integer> tableKeyIds = Arrays.asList(0, 2);
+    SamzaSqlCompositeKey compositeKey = SamzaSqlCompositeKey.createSamzaSqlCompositeKey(tableMsg, tableKeyIds);
+    KV<SamzaSqlCompositeKey, SamzaSqlRelMessage> record = KV.of(compositeKey, tableMsg);
+
+    SamzaSqlRelMessageJoinFunction joinFn =
+        new SamzaSqlRelMessageJoinFunction(joinRelType, false, streamKeyIds, streamFieldNames, tableFieldNames);
+    SamzaSqlRelMessage outMsg = joinFn.apply(streamMsg, record);
+
+    Assert.assertEquals(outMsg.getSamzaSqlRelRecord().getFieldValues().size(),
+        outMsg.getSamzaSqlRelRecord().getFieldNames().size());
+    List<String> expectedFieldNames = new ArrayList<>(tableFieldNames);
+    expectedFieldNames.addAll(streamFieldNames);
+    List<Object> expectedFieldValues = new ArrayList<>(tableFieldValues);
+    expectedFieldValues.addAll(streamFieldValues);
+    Assert.assertEquals(outMsg.getSamzaSqlRelRecord().getFieldValues(), expectedFieldValues);
+  }
+
+  @Test
+  public void testNullRecordWithInnerJoin() {
+    SamzaSqlRelMessage streamMsg = new SamzaSqlRelMessage(streamFieldNames, streamFieldValues);
+    JoinRelType joinRelType = JoinRelType.INNER;
+    List<Integer> streamKeyIds = Arrays.asList(0, 1);
+
+    SamzaSqlRelMessageJoinFunction joinFn =
+        new SamzaSqlRelMessageJoinFunction(joinRelType, true, streamKeyIds, streamFieldNames, tableFieldNames);
+    SamzaSqlRelMessage outMsg = joinFn.apply(streamMsg, null);
+    Assert.assertNull(outMsg);
+  }
+
+  @Test
+  public void testNullRecordWithLeftOuterJoin() {
+    SamzaSqlRelMessage streamMsg = new SamzaSqlRelMessage(streamFieldNames, streamFieldValues);
+    JoinRelType joinRelType = JoinRelType.LEFT;
+    List<Integer> streamKeyIds = Arrays.asList(0, 1);
+
+    SamzaSqlRelMessageJoinFunction joinFn =
+        new SamzaSqlRelMessageJoinFunction(joinRelType, true, streamKeyIds, streamFieldNames,
+            tableFieldNames);
+    SamzaSqlRelMessage outMsg = joinFn.apply(streamMsg, null);
+
+    Assert.assertEquals(outMsg.getSamzaSqlRelRecord().getFieldValues().size(),
+        outMsg.getSamzaSqlRelRecord().getFieldNames().size());
+    List<String> expectedFieldNames = new ArrayList<>(streamFieldNames);
+    expectedFieldNames.addAll(tableFieldNames);
+    List<Object> expectedFieldValues = new ArrayList<>(streamFieldValues);
+    expectedFieldValues.addAll(tableFieldNames.stream().map( name -> null ).collect(Collectors.toList()));
+    Assert.assertEquals(outMsg.getSamzaSqlRelRecord().getFieldValues(), expectedFieldValues);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-sql/src/test/java/org/apache/samza/sql/translator/TranslatorTestBase.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TranslatorTestBase.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TranslatorTestBase.java
new file mode 100644
index 0000000..a74993f
--- /dev/null
+++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TranslatorTestBase.java
@@ -0,0 +1,72 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*   http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied.  See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package org.apache.samza.sql.translator;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.TableDescriptor;
+import org.apache.samza.operators.TableImpl;
+import org.apache.samza.serializers.JsonSerdeV2;
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.sql.data.SamzaSqlRelMessage;
+import org.apache.samza.storage.kv.RocksDbTableProvider;
+import org.apache.samza.table.Table;
+import org.apache.samza.table.TableSpec;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import static org.mockito.Mockito.*;
+
+
+/**
+ * Base class for all unit tests for translators
+ */
+public class TranslatorTestBase {
+  Map<Integer, MessageStream> registeredStreams = new HashMap<>();
+  Map<String, TableImpl> registeredTables = new HashMap<>();
+
+  Answer getRegisterMessageStreamAnswer() {
+    return (InvocationOnMock x) -> {
+      Integer id = x.getArgumentAt(0, Integer.class);
+      MessageStream stream = x.getArgumentAt(1, MessageStream.class);
+      registeredStreams.put(id, stream);
+      return null;
+    };
+  }
+
+  Answer getRegisteredTableAnswer() {
+    return (InvocationOnMock x) -> {
+      TableDescriptor descriptor = x.getArgumentAt(0, TableDescriptor.class);
+      TableSpec mockTableSpec = new TableSpec(descriptor.getTableId(), KVSerde.of(new StringSerde(),
+          new JsonSerdeV2<SamzaSqlRelMessage>()), RocksDbTableProvider.class.getCanonicalName(), new HashMap<>());
+      TableImpl mockTable = mock(TableImpl.class);
+      when(mockTable.getTableSpec()).thenReturn(mockTableSpec);
+      this.registeredTables.putIfAbsent(descriptor.getTableId(), mockTable);
+      return this.registeredTables.get(descriptor.getTableId());
+    };
+  }
+
+  MessageStream getRegisteredMessageStream(int id) {
+    return this.registeredStreams.get(id);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/53d7f262/samza-test/src/main/java/org/apache/samza/example/AppWithGlobalConfigExample.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/example/AppWithGlobalConfigExample.java b/samza-test/src/main/java/org/apache/samza/example/AppWithGlobalConfigExample.java
new file mode 100644
index 0000000..c029eb4
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/example/AppWithGlobalConfigExample.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.example;
+
+import java.time.Duration;
+import org.apache.samza.application.StreamApplication;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.KV;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.triggers.Triggers;
+import org.apache.samza.operators.windows.AccumulationMode;
+import org.apache.samza.operators.windows.WindowPane;
+import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.runtime.LocalApplicationRunner;
+import org.apache.samza.serializers.JsonSerdeV2;
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.util.CommandLine;
+
+
+/**
+ * Example code to implement window-based counter
+ */
+public class AppWithGlobalConfigExample implements StreamApplication {
+
+  // local execution mode
+  public static void main(String[] args) {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+    LocalApplicationRunner runner = new LocalApplicationRunner(config);
+    AppWithGlobalConfigExample app = new AppWithGlobalConfigExample();
+    runner.run(app);
+    runner.waitForFinish();
+  }
+
+  @Override
+  public void init(StreamGraph graph, Config config) {
+    graph.getInputStream("myPageViewEevent", KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewEvent.class)))
+        .map(KV::getValue)
+        .window(Windows.<PageViewEvent, String, Integer>keyedTumblingWindow(m -> m.memberId, Duration.ofSeconds(10), () -> 0, (m, c) -> c + 1, null, null)
+            .setEarlyTrigger(Triggers.repeat(Triggers.count(5)))
+            .setAccumulationMode(AccumulationMode.DISCARDING), "w1")
+        .map(m -> KV.of(m.getKey().getKey(), new PageViewCount(m)))
+        .sendTo(graph.getOutputStream("pageViewEventPerMemberStream", KVSerde.of(new StringSerde("UTF-8"), new JsonSerdeV2<>(PageViewCount.class))));
+  }
+
+  class PageViewEvent {
+    String pageId;
+    String memberId;
+    long timestamp;
+
+    PageViewEvent(String pageId, String memberId, long timestamp) {
+      this.pageId = pageId;
+      this.memberId = memberId;
+      this.timestamp = timestamp;
+    }
+  }
+
+  static class PageViewCount {
+    String memberId;
+    long timestamp;
+    int count;
+
+    PageViewCount(WindowPane<String, Integer> m) {
+      this.memberId = m.getKey().getKey();
+      this.timestamp = Long.valueOf(m.getKey().getPaneId());
+      this.count = m.getMessage();
+    }
+  }
+}