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 2017/02/16 18:40:44 UTC

[01/14] samza git commit: SAMZA-1082 : Implement Leader Election using ZK

Repository: samza
Updated Branches:
  refs/heads/samza-fluent-api-v1 001be632d -> 5d416cfc4


SAMZA-1082 : Implement Leader Election using ZK

Simple implementation of leader election recipe along with unit tests

Author: navina <na...@apache.org>

Reviewers: Xinyu Liu <xi...@gmail.com>, Fred Ji <fj...@linkedin.com>

Closes #48 from navina/LeaderElector


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

Branch: refs/heads/samza-fluent-api-v1
Commit: 2226e3e710171336e0904c5a097d8ac8fff9f517
Parents: 38b1dc3
Author: navina <na...@apache.org>
Authored: Mon Feb 13 13:32:06 2017 -0800
Committer: navina <na...@apache.org>
Committed: Mon Feb 13 13:32:06 2017 -0800

----------------------------------------------------------------------
 build.gradle                                    |   1 +
 gradle/dependency-versions.gradle               |   2 +-
 .../java/org/apache/samza/config/ZkConfig.java  |  49 +++
 .../leaderelection/LeaderElector.java           |  46 +++
 .../java/org/apache/samza/zk/ZkKeyBuilder.java  |  74 ++++
 .../org/apache/samza/zk/ZkLeaderElector.java    | 162 ++++++++
 .../main/java/org/apache/samza/zk/ZkUtils.java  | 146 +++++++
 .../samza/coordinator/JobModelManager.scala     |   2 -
 .../samza/testUtils/EmbeddedZookeeper.java      | 103 +++++
 .../org/apache/samza/testUtils/FileUtil.java    |  81 ++++
 .../org/apache/samza/zk/TestZkKeyBuilder.java   |  53 +++
 .../apache/samza/zk/TestZkLeaderElector.java    | 405 +++++++++++++++++++
 .../java/org/apache/samza/zk/TestZkUtils.java   | 105 +++++
 13 files changed, 1226 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/2226e3e7/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 5b41c52..0d60970 100644
--- a/build.gradle
+++ b/build.gradle
@@ -159,6 +159,7 @@ project(":samza-core_$scalaVersion") {
     compile "net.sf.jopt-simple:jopt-simple:$joptSimpleVersion"
     compile "org.codehaus.jackson:jackson-mapper-asl:$jacksonVersion"
     compile "org.eclipse.jetty:jetty-webapp:$jettyVersion"
+    compile "com.101tec:zkclient:$zkClientVersion"
     testCompile "junit:junit:$junitVersion"
     testCompile "org.mockito:mockito-all:$mockitoVersion"
     testCompile "org.scalatest:scalatest_$scalaVersion:$scalaTestVersion"

http://git-wip-us.apache.org/repos/asf/samza/blob/2226e3e7/gradle/dependency-versions.gradle
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle
index db59672..0193b64 100644
--- a/gradle/dependency-versions.gradle
+++ b/gradle/dependency-versions.gradle
@@ -26,7 +26,7 @@
   mockitoVersion = "1.8.4"
   scalaTestVersion = "2.2.4"
   zkClientVersion = "0.8"
-  zookeeperVersion = "3.3.4"
+  zookeeperVersion = "3.4.6"
   metricsVersion = "2.2.0"
   kafkaVersion = "0.10.0.1"
   commonsHttpClientVersion = "3.1"

http://git-wip-us.apache.org/repos/asf/samza/blob/2226e3e7/samza-core/src/main/java/org/apache/samza/config/ZkConfig.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/config/ZkConfig.java b/samza-core/src/main/java/org/apache/samza/config/ZkConfig.java
new file mode 100644
index 0000000..f26b2d9
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/config/ZkConfig.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.config;
+
+public class ZkConfig extends MapConfig {
+  // Connection string for ZK, format: :<hostname>:<port>,..."
+  public static final String ZK_CONNECT = "coordinator.zk.connect";
+  public static final String ZK_SESSION_TIMEOUT_MS = "coordinator.zk.session-timeout-ms";
+  public static final String ZK_CONNECTION_TIMEOUT_MS = "coordinator.zk.connection-timeout-ms";
+
+  public static final int DEFAULT_CONNECTION_TIMEOUT_MS = 60000;
+  public static final int DEFAULT_SESSION_TIMEOUT_MS = 30000;
+
+  public ZkConfig(Config config) {
+    super(config);
+  }
+
+  public String getZkConnect() {
+    if (!containsKey(ZK_CONNECT)) {
+      throw new ConfigException("Missing " + ZK_CONNECT + " config!");
+    }
+    return get(ZK_CONNECT);
+  }
+
+  public int getZkSessionTimeoutMs() {
+    return getInt(ZK_SESSION_TIMEOUT_MS, DEFAULT_SESSION_TIMEOUT_MS);
+  }
+
+  public int getZkConnectionTimeoutMs() {
+    return getInt(ZK_CONNECTION_TIMEOUT_MS, DEFAULT_CONNECTION_TIMEOUT_MS);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/2226e3e7/samza-core/src/main/java/org/apache/samza/coordinator/leaderelection/LeaderElector.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/coordinator/leaderelection/LeaderElector.java b/samza-core/src/main/java/org/apache/samza/coordinator/leaderelection/LeaderElector.java
new file mode 100644
index 0000000..94e3311
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/coordinator/leaderelection/LeaderElector.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.coordinator.leaderelection;
+
+import org.apache.samza.annotation.InterfaceStability;
+
+@InterfaceStability.Evolving
+public interface LeaderElector {
+  /**
+   * Method that helps the caller participate in leader election and returns when the participation is complete
+   *
+   * @return True, if caller is chosen as a leader through the leader election process. False, otherwise.
+   */
+  boolean tryBecomeLeader();
+
+  /**
+   * Method that allows a caller to resign from leadership role. Caller can resign from leadership due to various
+   * reasons such as shutdown, connection failures etc.
+   * This method should clear any state created by the leader and clean-up the resources used by the leader.
+   */
+  void resignLeadership();
+
+  /**
+   * Method that can be used to know if the caller is the current leader or not
+   *
+   * @return True, if the caller is the current leader. False, otherwise
+   */
+  boolean amILeader();
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/2226e3e7/samza-core/src/main/java/org/apache/samza/zk/ZkKeyBuilder.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkKeyBuilder.java b/samza-core/src/main/java/org/apache/samza/zk/ZkKeyBuilder.java
new file mode 100644
index 0000000..28344e9
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkKeyBuilder.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.zk;
+
+import com.google.common.base.Strings;
+import org.apache.samza.SamzaException;
+
+/**
+ * The following ZK hierarchy is maintained for Standalone jobs:
+ * <pre>
+ *   - /
+ *      |- jobName-jobId/
+ *          |- processors/
+ *              |- 00000001
+ *              |- 00000002
+ *              |- ...
+ * </pre>
+ * Note: ZK Node levels without an ending forward slash ('/') represent a leaf node and non-leaf node, otherwise.
+ *
+ * This class provides helper methods to easily generate/parse the path in the ZK hierarchy.
+ */
+public class ZkKeyBuilder {
+  /**
+   * Prefix generated to uniquely identify a particular deployment of a job.
+   * TODO: For now, it looks like $jobName-$jobId. We need to add a unique deployment/attempt identifier as well.
+   */
+  private final String pathPrefix;
+
+  static final String PROCESSORS_PATH = "processors";
+  static final String PROCESSOR_ID_PREFIX = "processor-";
+
+  public ZkKeyBuilder(String pathPrefix) {
+    if (Strings.isNullOrEmpty(pathPrefix)) {
+      throw new SamzaException("Zk PathPrefix cannot be null or empty!");
+    }
+    this.pathPrefix = pathPrefix.trim();
+  }
+
+  public String getProcessorsPath() {
+    return String.format("/%s/%s", pathPrefix, PROCESSORS_PATH);
+  }
+
+  /**
+   * Static method that helps parse the processorId substring from the ZK path
+   *
+   * Processor ID is prefixed by "processor-" and is an leaf node in ZK tree. Hence, this pattern is used to extract
+   * the processorId.
+   *
+   * @param path Full ZK path of a registered processor
+   * @return String representing the processor ID
+   */
+  public static String parseIdFromPath(String path) {
+    if (!Strings.isNullOrEmpty(path))
+      return path.substring(path.lastIndexOf("/") + 1);
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/2226e3e7/samza-core/src/main/java/org/apache/samza/zk/ZkLeaderElector.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkLeaderElector.java b/samza-core/src/main/java/org/apache/samza/zk/ZkLeaderElector.java
new file mode 100644
index 0000000..8cdf8fc
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkLeaderElector.java
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.zk;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.I0Itec.zkclient.IZkDataListener;
+import org.apache.samza.SamzaException;
+import org.apache.samza.coordinator.leaderelection.LeaderElector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * <p>
+ * An implementation of Leader Elector using Zookeeper.
+ *
+ * Each participant in the leader election process creates an instance of this class and tries to become the leader.
+ * The participant with the lowest sequence number in the ZK subtree for election becomes the leader. Every non-leader
+ * sets a watcher on its predecessor, where the predecessor is the participant with the largest sequence number
+ * that is less than the current participant's sequence number.
+ * </p>
+ * */
+public class ZkLeaderElector implements LeaderElector {
+  public static final Logger LOGGER = LoggerFactory.getLogger(ZkLeaderElector.class);
+  private final ZkUtils zkUtils;
+  private final String processorIdStr;
+  private final ZkKeyBuilder keyBuilder;
+  private final String hostName;
+
+  private AtomicBoolean isLeader = new AtomicBoolean(false);
+  private final IZkDataListener zkLeaderElectionListener;
+  private String currentSubscription = null;
+  private final Random random = new Random();
+
+  @VisibleForTesting
+  ZkLeaderElector(String processorIdStr, ZkUtils zkUtils, IZkDataListener leaderElectionListener) {
+    this.processorIdStr = processorIdStr;
+    this.zkUtils = zkUtils;
+    this.zkLeaderElectionListener = leaderElectionListener;
+    this.keyBuilder = this.zkUtils.getKeyBuilder();
+    this.hostName = getHostName();
+  }
+
+  public ZkLeaderElector(String processorIdStr, ZkUtils zkUtils) {
+    this.zkLeaderElectionListener = new ZkLeaderElectionListener();
+    this.processorIdStr = processorIdStr;
+    this.zkUtils = zkUtils;
+    this.keyBuilder = this.zkUtils.getKeyBuilder();
+    this.hostName = getHostName();
+  }
+
+  // TODO: This should go away once we integrate with Zk based Job Coordinator
+  private String getHostName() {
+    try {
+      return InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException e) {
+      LOGGER.error("Failed to fetch hostname of the processor", e);
+      throw new SamzaException(e);
+    }
+  }
+
+  @Override
+  public boolean tryBecomeLeader() {
+    String currentPath = zkUtils.registerProcessorAndGetId(hostName);
+
+    List<String> children = zkUtils.getSortedActiveProcessors();
+    LOGGER.debug(zLog("Current active processors - " + children));
+    int index = children.indexOf(ZkKeyBuilder.parseIdFromPath(currentPath));
+
+    if (children.size() == 0 || index == -1) {
+      throw new SamzaException("Looks like we are no longer connected to Zk. Need to reconnect!");
+    }
+
+    if (index == 0) {
+      isLeader.getAndSet(true);
+      LOGGER.info(zLog("Eligible to become the leader!"));
+      return true;
+    }
+
+    isLeader.getAndSet(false);
+    LOGGER.info("Index = " + index + " Not eligible to be a leader yet!");
+    String predecessor = children.get(index - 1);
+    if (!predecessor.equals(currentSubscription)) {
+      if (currentSubscription != null) {
+        LOGGER.debug(zLog("Unsubscribing data change for " + currentSubscription));
+        zkUtils.unsubscribeDataChanges(keyBuilder.getProcessorsPath() + "/" + currentSubscription, zkLeaderElectionListener);
+      }
+      currentSubscription = predecessor;
+      LOGGER.info(zLog("Subscribing data change for " + predecessor));
+      zkUtils.subscribeDataChanges(keyBuilder.getProcessorsPath() + "/" + currentSubscription, zkLeaderElectionListener);
+    }
+    /**
+     * Verify that the predecessor still exists. This step is needed because the ZkClient subscribes for data changes
+     * on the path, even if the path doesn't exist. Since we are using Ephemeral Sequential nodes, if the path doesn't
+     * exist during subscription, it is not going to get created in the future.
+     */
+    boolean predecessorExists = zkUtils.exists(keyBuilder.getProcessorsPath() + "/" + currentSubscription);
+    if (predecessorExists) {
+      LOGGER.info(zLog("Predecessor still exists. Current subscription is valid. Continuing as non-leader."));
+    } else {
+      try {
+        Thread.sleep(random.nextInt(1000));
+      } catch (InterruptedException e) {
+        Thread.interrupted();
+      }
+      LOGGER.info(zLog("Predecessor doesn't exist anymore. Trying to become leader again..."));
+      return tryBecomeLeader();
+    }
+    return false;
+  }
+
+  @Override
+  public void resignLeadership() {
+    isLeader.compareAndSet(true, false);
+  }
+
+  @Override
+  public boolean amILeader() {
+    return isLeader.get();
+  }
+
+  private String zLog(String logMessage) {
+    return String.format("[Processor-%s] %s", processorIdStr, logMessage);
+  }
+
+  // Only by non-leaders
+  class ZkLeaderElectionListener implements IZkDataListener {
+
+    @Override
+    public void handleDataChange(String dataPath, Object data) throws Exception {
+      LOGGER.debug("Data change on path: " + dataPath + " Data: " + data);
+    }
+
+    @Override
+    public void handleDataDeleted(String dataPath) throws Exception {
+      LOGGER.info(zLog("Data deleted on path " + dataPath + ". Predecessor went away. So, trying to become leader again..."));
+      tryBecomeLeader();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/2226e3e7/samza-core/src/main/java/org/apache/samza/zk/ZkUtils.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkUtils.java b/samza-core/src/main/java/org/apache/samza/zk/ZkUtils.java
new file mode 100644
index 0000000..d0a269d
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkUtils.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.zk;
+
+import org.I0Itec.zkclient.IZkDataListener;
+import org.I0Itec.zkclient.ZkClient;
+import org.I0Itec.zkclient.ZkConnection;
+import org.I0Itec.zkclient.exception.ZkInterruptedException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Util class to help manage Zk connection and ZkClient.
+ * It also provides additional utility methods for read/write/subscribe/unsubscribe access to the ZK tree.
+ *
+ * <p>
+ *  <b>Note on ZkClient:</b>
+ *  {@link ZkClient} consists of two threads - I/O thread and Event thread.
+ *  I/O thread manages heartbeats to the Zookeeper server in the ensemble and handles responses to synchronous methods
+ *  in Zookeeper API.
+ *  Event thread typically receives all the Watcher events and delivers to registered listeners. It, also, handles
+ *  responses to asynchronous methods in Zookeeper API.
+ * </p>
+ *
+ * <p>
+ *   <b>Note on Session Management:</b>
+ *   Session management, if needed, should be handled by the caller. This can be done by implementing
+ *   {@link org.I0Itec.zkclient.IZkStateListener} and subscribing this listener to the current ZkClient. Note: The connection state change
+ *   callbacks are invoked in the context of the Event thread of the ZkClient. So, it is advised to do non-blocking
+ *   processing in the callbacks.
+ * </p>
+ */
+public class ZkUtils {
+  private static final Logger LOG = LoggerFactory.getLogger(ZkUtils.class);
+
+  private final ZkClient zkClient;
+  private volatile String ephemeralPath = null;
+  private final ZkKeyBuilder keyBuilder;
+  private final int connectionTimeoutMs;
+
+  public ZkUtils(ZkKeyBuilder zkKeyBuilder, ZkClient zkClient, int connectionTimeoutMs) {
+    this.keyBuilder = zkKeyBuilder;
+    this.connectionTimeoutMs = connectionTimeoutMs;
+    this.zkClient = zkClient;
+  }
+
+  public void connect() throws ZkInterruptedException {
+    boolean isConnected = zkClient.waitUntilConnected(connectionTimeoutMs, TimeUnit.MILLISECONDS);
+    if (!isConnected) {
+      throw new RuntimeException("Unable to connect to Zookeeper within connectionTimeout " + connectionTimeoutMs + "ms. Shutting down!");
+    }
+  }
+
+  public static ZkConnection createZkConnection(String zkConnectString, int sessionTimeoutMs) {
+    return new ZkConnection(zkConnectString, sessionTimeoutMs);
+  }
+
+  public static ZkClient createZkClient(ZkConnection zkConnection, int connectionTimeoutMs) {
+    return new ZkClient(zkConnection, connectionTimeoutMs);
+  }
+
+  ZkClient getZkClient() {
+    return zkClient;
+  }
+
+  public ZkKeyBuilder getKeyBuilder() {
+    return keyBuilder;
+  }
+
+  /**
+   * Returns a ZK generated identifier for this client.
+   * If the current client is registering for the first time, it creates an ephemeral sequential node in the ZK tree
+   * If the current client has already registered and is still within the same session, it returns the already existing
+   * value for the ephemeralPath
+   *
+   * @param data Object that should be written as data in the registered ephemeral ZK node
+   * @return String representing the absolute ephemeralPath of this client in the current session
+   */
+  public synchronized String registerProcessorAndGetId(final Object data) {
+    if (ephemeralPath == null) {
+      // TODO: Data should be more than just the hostname. Use Json serialized data
+      ephemeralPath =
+          zkClient.createEphemeralSequential(
+              keyBuilder.getProcessorsPath() + "/", data);
+      return ephemeralPath;
+    } else {
+      return ephemeralPath;
+    }
+  }
+
+  public synchronized String getEphemeralPath() {
+    return ephemeralPath;
+  }
+
+  /**
+   * Method is used to get the <i>sorted</i> list of currently active/registered processors
+   *
+   * @return List of absolute ZK node paths
+   */
+  public List<String> getSortedActiveProcessors() {
+    List<String> children = zkClient.getChildren(keyBuilder.getProcessorsPath());
+    if (children.size() > 0) {
+      Collections.sort(children);
+      LOG.info("Found these children - " + children);
+    }
+    return children;
+  }
+
+  /* Wrapper for standard I0Itec methods */
+  public void unsubscribeDataChanges(String path, IZkDataListener dataListener) {
+    zkClient.unsubscribeDataChanges(path, dataListener);
+  }
+
+  public void subscribeDataChanges(String path, IZkDataListener dataListener) {
+    zkClient.subscribeDataChanges(path, dataListener);
+  }
+
+  public boolean exists(String path) {
+    return zkClient.exists(path);
+  }
+
+  public void close() throws ZkInterruptedException {
+    zkClient.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/2226e3e7/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala b/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala
index 85f4df0..7f5d05d 100644
--- a/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala
+++ b/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala
@@ -214,7 +214,6 @@ object JobModelManager extends Logging {
 
     // Generate the jobModel
     def jobModelGenerator(): JobModel = refreshJobModel(config,
-                                                        allSystemStreamPartitions,
                                                         groups,
                                                         previousChangelogMapping,
                                                         localityManager)
@@ -247,7 +246,6 @@ object JobModelManager extends Logging {
    * refresh. Hence, there is no need for synchronization as before.
    */
   private def refreshJobModel(config: Config,
-                              allSystemStreamPartitions: util.Set[SystemStreamPartition],
                               groups: util.Map[TaskName, util.Set[SystemStreamPartition]],
                               previousChangelogMapping: util.Map[TaskName, Integer],
                               localityManager: LocalityManager): JobModel = {

http://git-wip-us.apache.org/repos/asf/samza/blob/2226e3e7/samza-core/src/test/java/org/apache/samza/testUtils/EmbeddedZookeeper.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/testUtils/EmbeddedZookeeper.java b/samza-core/src/test/java/org/apache/samza/testUtils/EmbeddedZookeeper.java
new file mode 100644
index 0000000..bd0a2d1
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/testUtils/EmbeddedZookeeper.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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;
+
+import org.apache.zookeeper.server.NIOServerCnxnFactory;
+import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.apache.zookeeper.server.ZooKeeperServer;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+public class EmbeddedZookeeper {
+  private static final Logger LOGGER = LoggerFactory.getLogger(EmbeddedZookeeper.class);
+
+  private static final String SNAPSHOT_DIR_RELATIVE_PATH = "zk/snapshot";
+  private static final String LOG_DIR_RELATIVE_PATH = "zk/log";
+  private static final int TICK_TIME = 500;
+  private static final int MAX_CLIENT_CONNECTIONS = 1024;
+  private static final int RANDOM_PORT = 0;
+
+  private ZooKeeperServer zooKeeperServer = null;
+  private ServerCnxnFactory serverCnxnFactory = null;
+  private File snapshotDir = null;
+  private File logDir = null;
+
+  public void setup() {
+    try {
+      snapshotDir = FileUtil.createFileInTempDir(SNAPSHOT_DIR_RELATIVE_PATH);
+      logDir = FileUtil.createFileInTempDir(LOG_DIR_RELATIVE_PATH);
+    } catch (IOException e) {
+      LOGGER.error("Failed to setup Zookeeper Server Environment", e);
+      Assert.fail("Failed to setup Zookeeper Server Environment");
+    }
+
+    try {
+      zooKeeperServer = new ZooKeeperServer(snapshotDir, logDir, TICK_TIME);
+      serverCnxnFactory = NIOServerCnxnFactory.createFactory();
+      InetSocketAddress addr = new InetSocketAddress("127.0.0.1", RANDOM_PORT);
+      serverCnxnFactory.configure(addr, MAX_CLIENT_CONNECTIONS);
+
+      serverCnxnFactory.startup(zooKeeperServer);
+    } catch (Exception e) {
+      LOGGER.error("Zookeeper Server failed to start", e);
+      Assert.fail("Zookeeper Server failed to start");
+    }
+  }
+
+  public void teardown() {
+    serverCnxnFactory.shutdown();
+
+    try {
+      serverCnxnFactory.join();
+    } catch (InterruptedException e) {
+      LOGGER.warn("Zookeeper server may not have terminated cleanly!", e);
+    }
+
+    try {
+      FileUtil.deleteDir(snapshotDir);
+      FileUtil.deleteDir(logDir);
+    } catch (FileNotFoundException | NullPointerException e) {
+      LOGGER.warn("Zookeeper Server Environment Cleanup failed!", e);
+    }
+  }
+
+  public int getPort() {
+    return zooKeeperServer.getClientPort();
+  }
+
+  public static void main(String[] args) {
+    EmbeddedZookeeper zk = new EmbeddedZookeeper();
+    zk.setup();
+    System.out.println("Zk Server Started!!");
+    try {
+      Thread.sleep(10000);
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    }
+
+    zk.teardown();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/2226e3e7/samza-core/src/test/java/org/apache/samza/testUtils/FileUtil.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/testUtils/FileUtil.java b/samza-core/src/test/java/org/apache/samza/testUtils/FileUtil.java
new file mode 100644
index 0000000..b33a9fa
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/testUtils/FileUtil.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.testUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+public class FileUtil {
+  public static final String TMP_DIR = System.getProperty("java.io.tmpdir");
+  private static final Logger LOGGER = LoggerFactory.getLogger(FileUtil.class);
+
+  private FileUtil() {}
+
+  /**
+   * Creates a file, along with any parents (if any), in the system-specific Java temporary directory
+   * If the file already exists, this method simply returns
+   *
+   * @param path Path relative to the temporary directory
+   * @return True, if the file was created successfully, along with parent files (if any)
+   * @throws IOException
+   */
+  static File createFileInTempDir(String path) throws IOException {
+    if (path == null || path.isEmpty()) {
+      throw new RuntimeException("Unable to create file - Null or empty path!");
+    }
+    File file = new File(TMP_DIR, path);
+    if (!file.exists()) {
+      if (!file.mkdirs()) {
+        throw new IOException("Failed to create file");
+      }
+    }
+    return file;
+  }
+
+  /**
+   * Deletes a given {@link File}, if it exists. If it doesn't exist, it throws a {@link FileNotFoundException}
+   * If the given {@link File} is a directory, it recursively deletes the files in the directory, before deleting the
+   * directory itself.
+   *
+   * @param path Reference to the {@link File} to be deleted
+   * @return True, if it successfully deleted the given {@link File}. False, otherwise.
+   * @throws FileNotFoundException When the given {@link File} does not exist
+   * @throws NullPointerException When the given {@link File} reference is null
+   */
+  static boolean deleteDir(File path) throws FileNotFoundException, NullPointerException {
+    if (path == null) {
+      throw new NullPointerException("Path cannot be null!");
+    }
+    if (!path.exists()) {
+      throw new FileNotFoundException("File not found: " + path);
+    }
+    boolean result = true;
+
+    if (path.isDirectory()) {
+      for (File f: path.listFiles()) {
+        result = result & deleteDir(f);
+      }
+    }
+    return result && path.delete();
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/2226e3e7/samza-core/src/test/java/org/apache/samza/zk/TestZkKeyBuilder.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/zk/TestZkKeyBuilder.java b/samza-core/src/test/java/org/apache/samza/zk/TestZkKeyBuilder.java
new file mode 100644
index 0000000..e04f7c9
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/zk/TestZkKeyBuilder.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.zk;
+
+import org.apache.samza.SamzaException;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZkKeyBuilder {
+
+  @Test
+  public void pathPrefixCannotBeNullOrEmpty() {
+    try {
+      new ZkKeyBuilder("");
+      Assert.fail("Key Builder was created with empty path prefix!");
+      new ZkKeyBuilder(null);
+      Assert.fail("Key Builder was created with null path prefix!");
+    } catch (SamzaException e) {
+      // Expected
+    }
+  }
+
+  @Test
+  public void testProcessorsPath() {
+    ZkKeyBuilder builder = new ZkKeyBuilder("test");
+    Assert.assertEquals("/test/" + ZkKeyBuilder.PROCESSORS_PATH, builder.getProcessorsPath());
+  }
+
+  @Test
+  public void testParseIdFromPath() {
+    Assert.assertEquals(
+        ZkKeyBuilder.PROCESSOR_ID_PREFIX + "1",
+        ZkKeyBuilder.parseIdFromPath("/test/processors/" + ZkKeyBuilder.PROCESSOR_ID_PREFIX + "1"));
+    Assert.assertNull(ZkKeyBuilder.parseIdFromPath(null));
+    Assert.assertNull(ZkKeyBuilder.parseIdFromPath(""));
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/2226e3e7/samza-core/src/test/java/org/apache/samza/zk/TestZkLeaderElector.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/zk/TestZkLeaderElector.java b/samza-core/src/test/java/org/apache/samza/zk/TestZkLeaderElector.java
new file mode 100644
index 0000000..b999ec5
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/zk/TestZkLeaderElector.java
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.zk;
+
+import org.I0Itec.zkclient.IZkDataListener;
+import org.I0Itec.zkclient.ZkConnection;
+import org.I0Itec.zkclient.exception.ZkNodeExistsException;
+import org.apache.samza.SamzaException;
+import org.apache.samza.testUtils.EmbeddedZookeeper;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class TestZkLeaderElector {
+
+  private static EmbeddedZookeeper zkServer = null;
+  private static final ZkKeyBuilder KEY_BUILDER = new ZkKeyBuilder("test");
+  private String testZkConnectionString = null;
+  private ZkUtils testZkUtils = null;
+  private static final int SESSION_TIMEOUT_MS = 20000;
+  private static final int CONNECTION_TIMEOUT_MS = 10000;
+
+  @BeforeClass
+  public static void setup() throws InterruptedException {
+    zkServer = new EmbeddedZookeeper();
+    zkServer.setup();
+  }
+
+  @Before
+  public void testSetup() {
+    testZkConnectionString = "localhost:" + zkServer.getPort();
+    try {
+      testZkUtils = getZkUtilsWithNewClient();
+    } catch (Exception e) {
+      Assert.fail("Client connection setup failed. Aborting tests..");
+    }
+    try {
+      testZkUtils.getZkClient().createPersistent(KEY_BUILDER.getProcessorsPath(), true);
+    } catch (ZkNodeExistsException e) {
+      // Do nothing
+    }
+  }
+
+
+  @After
+  public void testTeardown() {
+    testZkUtils.close();
+  }
+
+  @AfterClass
+  public static void teardown() {
+    zkServer.teardown();
+  }
+
+  @Test
+  public void testLeaderElectionRegistersProcessor() {
+    List<String> activeProcessors = new ArrayList<String>() {
+      {
+        add("0000000000");
+      }
+    };
+
+    ZkUtils mockZkUtils = mock(ZkUtils.class);
+    when(mockZkUtils.registerProcessorAndGetId(any())).
+        thenReturn(KEY_BUILDER.getProcessorsPath() + "/0000000000");
+    when(mockZkUtils.getSortedActiveProcessors()).thenReturn(activeProcessors);
+
+    ZkLeaderElector leaderElector = new ZkLeaderElector("1", mockZkUtils);
+    Assert.assertTrue(leaderElector.tryBecomeLeader());
+  }
+
+  @Test
+  public void testUnregisteredProcessorInLeaderElection() {
+    String processorId = "1";
+    ZkUtils mockZkUtils = mock(ZkUtils.class);
+    when(mockZkUtils.getSortedActiveProcessors()).thenReturn(new ArrayList<String>());
+
+    ZkLeaderElector leaderElector = new ZkLeaderElector(processorId, mockZkUtils);
+    try {
+      leaderElector.tryBecomeLeader();
+      Assert.fail("Was expecting leader election to fail!");
+    } catch (SamzaException e) {
+      // No-op Expected
+    }
+  }
+
+  /**
+   * Test starts 3 processors and verifies the state of the Zk tree after all processors participate in LeaderElection
+   */
+  @Test
+  public void testLeaderElection() {
+    // Processor-1
+    ZkUtils zkUtils1 = getZkUtilsWithNewClient();
+    ZkLeaderElector leaderElector1 = new ZkLeaderElector(
+        "1",
+        zkUtils1);
+
+    // Processor-2
+    ZkUtils zkUtils2 = getZkUtilsWithNewClient();
+    ZkLeaderElector leaderElector2 = new ZkLeaderElector(
+        "2",
+        zkUtils2);
+
+    // Processor-3
+    ZkUtils zkUtils3  = getZkUtilsWithNewClient();
+    ZkLeaderElector leaderElector3 = new ZkLeaderElector(
+        "3",
+        zkUtils3);
+
+    Assert.assertEquals(0, testZkUtils.getSortedActiveProcessors().size());
+
+    Assert.assertTrue(leaderElector1.tryBecomeLeader());
+    Assert.assertFalse(leaderElector2.tryBecomeLeader());
+    Assert.assertFalse(leaderElector3.tryBecomeLeader());
+
+    Assert.assertEquals(3, testZkUtils.getSortedActiveProcessors().size());
+
+    // Clean up
+    zkUtils1.close();
+    zkUtils2.close();
+    zkUtils3.close();
+
+    Assert.assertEquals(new ArrayList<String>(), testZkUtils.getSortedActiveProcessors());
+
+  }
+
+  /**
+   * Tests that Leader Failure automatically promotes the next successor to become the leader
+   */
+  @Test
+  public void testLeaderFailure() {
+    /**
+     * electionLatch and count together verify that:
+     * 1. the registered listeners are actually invoked by the ZkClient on the correct path
+     * 2. for a single participant failure, at-most 1 other participant is notified
+     */
+    final CountDownLatch electionLatch = new CountDownLatch(1);
+    final AtomicInteger count = new AtomicInteger(0);
+
+    // Processor-1
+    ZkUtils zkUtils1 = getZkUtilsWithNewClient();
+    zkUtils1.registerProcessorAndGetId("processor1");
+    ZkLeaderElector leaderElector1 = new ZkLeaderElector(
+        "1",
+        zkUtils1,
+        new IZkDataListener() {
+          @Override
+          public void handleDataChange(String dataPath, Object data) throws Exception {
+
+          }
+
+          @Override
+          public void handleDataDeleted(String dataPath) throws Exception {
+            count.incrementAndGet();
+          }
+        });
+
+
+    // Processor-2
+    ZkUtils zkUtils2 = getZkUtilsWithNewClient();
+    final String path2 = zkUtils2.registerProcessorAndGetId("processor2");
+    ZkLeaderElector leaderElector2 = new ZkLeaderElector(
+        "2",
+        zkUtils2,
+        new IZkDataListener() {
+          @Override
+          public void handleDataChange(String dataPath, Object data) throws Exception {
+
+          }
+
+          @Override
+          public void handleDataDeleted(String dataPath) throws Exception {
+            String registeredIdStr = ZkKeyBuilder.parseIdFromPath(path2);
+            Assert.assertNotNull(registeredIdStr);
+
+            String predecessorIdStr = ZkKeyBuilder.parseIdFromPath(dataPath);
+            Assert.assertNotNull(predecessorIdStr);
+
+            try {
+              int selfId = Integer.parseInt(registeredIdStr);
+              int predecessorId = Integer.parseInt(predecessorIdStr);
+              Assert.assertEquals(1, selfId - predecessorId);
+            } catch (Exception e) {
+              System.out.println(e.getMessage());
+            }
+            count.incrementAndGet();
+            electionLatch.countDown();
+          }
+        });
+
+    // Processor-3
+    ZkUtils zkUtils3  = getZkUtilsWithNewClient();
+    zkUtils3.registerProcessorAndGetId("processor3");
+    ZkLeaderElector leaderElector3 = new ZkLeaderElector(
+        "3",
+        zkUtils3,
+        new IZkDataListener() {
+          @Override
+          public void handleDataChange(String dataPath, Object data) throws Exception {
+
+          }
+
+          @Override
+          public void handleDataDeleted(String dataPath) throws Exception {
+            count.incrementAndGet();
+          }
+        });
+
+    // Join Leader Election
+    Assert.assertTrue(leaderElector1.tryBecomeLeader());
+    Assert.assertFalse(leaderElector2.tryBecomeLeader());
+    Assert.assertFalse(leaderElector3.tryBecomeLeader());
+
+    Assert.assertTrue(leaderElector1.amILeader());
+    Assert.assertFalse(leaderElector2.amILeader());
+    Assert.assertFalse(leaderElector3.amILeader());
+
+    List<String> currentActiveProcessors = testZkUtils.getSortedActiveProcessors();
+    Assert.assertEquals(3, currentActiveProcessors.size());
+
+    // Leader Failure
+    zkUtils1.close();
+    currentActiveProcessors.remove(0);
+
+    try {
+      Assert.assertTrue(electionLatch.await(5, TimeUnit.SECONDS));
+    } catch (InterruptedException e) {
+      Assert.fail("Interrupted while waiting for leaderElection listener callback to complete!");
+    }
+
+    Assert.assertEquals(1, count.get());
+    Assert.assertEquals(currentActiveProcessors, testZkUtils.getSortedActiveProcessors());
+
+    // Clean up
+    zkUtils2.close();
+    zkUtils3.close();
+  }
+
+  /**
+   * Tests that a non-leader failure updates the Zk tree and participants' state correctly
+   */
+  @Test
+  public void testNonLeaderFailure() {
+    /**
+     * electionLatch and count together verify that:
+     * 1. the registered listeners are actually invoked by the ZkClient on the correct path
+     * 2. for a single participant failure, at-most 1 other participant is notified
+     */
+    final CountDownLatch electionLatch = new CountDownLatch(1);
+    final AtomicInteger count = new AtomicInteger(0);
+
+    // Processor-1
+    ZkUtils zkUtils1 = getZkUtilsWithNewClient();
+    zkUtils1.registerProcessorAndGetId("processor1");
+    ZkLeaderElector leaderElector1 = new ZkLeaderElector(
+        "1",
+        zkUtils1,
+        new IZkDataListener() {
+          @Override
+          public void handleDataChange(String dataPath, Object data) throws Exception {
+
+          }
+
+          @Override
+          public void handleDataDeleted(String dataPath) throws Exception {
+            count.incrementAndGet();
+          }
+        });
+
+    // Processor-2
+    ZkUtils zkUtils2 = getZkUtilsWithNewClient();
+    zkUtils2.registerProcessorAndGetId("processor2");
+    ZkLeaderElector leaderElector2 = new ZkLeaderElector(
+        "2",
+        zkUtils2,
+        new IZkDataListener() {
+          @Override
+          public void handleDataChange(String dataPath, Object data) throws Exception {
+
+          }
+
+          @Override
+          public void handleDataDeleted(String dataPath) throws Exception {
+            count.incrementAndGet();
+          }
+        });
+
+    // Processor-3
+    ZkUtils zkUtils3  = getZkUtilsWithNewClient();
+    final String path3 = zkUtils3.registerProcessorAndGetId("processor3");
+    ZkLeaderElector leaderElector3 = new ZkLeaderElector(
+        "3",
+        zkUtils3,
+        new IZkDataListener() {
+          @Override
+          public void handleDataChange(String dataPath, Object data) throws Exception {
+
+          }
+
+          @Override
+          public void handleDataDeleted(String dataPath) throws Exception {
+            String registeredIdStr = ZkKeyBuilder.parseIdFromPath(path3);
+            Assert.assertNotNull(registeredIdStr);
+
+            String predecessorIdStr = ZkKeyBuilder.parseIdFromPath(dataPath);
+            Assert.assertNotNull(predecessorIdStr);
+
+            try {
+              int selfId = Integer.parseInt(registeredIdStr);
+              int predecessorId = Integer.parseInt(predecessorIdStr);
+              Assert.assertEquals(1, selfId - predecessorId);
+            } catch (Exception e) {
+              Assert.fail("Exception in LeaderElectionListener!");
+            }
+            count.incrementAndGet();
+            electionLatch.countDown();
+          }
+        });
+
+    // Join Leader Election
+    Assert.assertTrue(leaderElector1.tryBecomeLeader());
+    Assert.assertFalse(leaderElector2.tryBecomeLeader());
+    Assert.assertFalse(leaderElector3.tryBecomeLeader());
+
+    List<String> currentActiveProcessors = testZkUtils.getSortedActiveProcessors();
+    Assert.assertEquals(3, currentActiveProcessors.size());
+
+    zkUtils2.close();
+    currentActiveProcessors.remove(1);
+
+    try {
+      Assert.assertTrue(electionLatch.await(5, TimeUnit.SECONDS));
+    } catch (InterruptedException e) {
+      Assert.fail("Interrupted while waiting for leaderElection listener callback to complete!");
+    }
+
+    Assert.assertEquals(1, count.get());
+    Assert.assertEquals(currentActiveProcessors, testZkUtils.getSortedActiveProcessors());
+
+    // Clean up
+    zkUtils1.close();
+    zkUtils3.close();
+  }
+
+  @Test
+  public void testAmILeader() {
+    // Processor-1
+    ZkLeaderElector leaderElector1 = new ZkLeaderElector(
+        "1",
+        getZkUtilsWithNewClient());
+
+    // Processor-2
+    ZkLeaderElector leaderElector2 = new ZkLeaderElector(
+        "2",
+        getZkUtilsWithNewClient());
+
+    // Before Leader Election
+    Assert.assertFalse(leaderElector1.amILeader());
+    Assert.assertFalse(leaderElector2.amILeader());
+
+    leaderElector1.tryBecomeLeader();
+    leaderElector2.tryBecomeLeader();
+
+    // After Leader Election
+    Assert.assertTrue(leaderElector1.amILeader());
+    Assert.assertFalse(leaderElector2.amILeader());
+  }
+
+  private ZkUtils getZkUtilsWithNewClient() {
+    ZkConnection zkConnection = ZkUtils.createZkConnection(testZkConnectionString, SESSION_TIMEOUT_MS);
+    return new ZkUtils(
+        KEY_BUILDER,
+        ZkUtils.createZkClient(zkConnection, CONNECTION_TIMEOUT_MS),
+        CONNECTION_TIMEOUT_MS);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/2226e3e7/samza-core/src/test/java/org/apache/samza/zk/TestZkUtils.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/zk/TestZkUtils.java b/samza-core/src/test/java/org/apache/samza/zk/TestZkUtils.java
new file mode 100644
index 0000000..855d29d
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/zk/TestZkUtils.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.zk;
+
+import org.I0Itec.zkclient.ZkClient;
+import org.I0Itec.zkclient.ZkConnection;
+import org.I0Itec.zkclient.exception.ZkNodeExistsException;
+import org.apache.samza.testUtils.EmbeddedZookeeper;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestZkUtils {
+  private static EmbeddedZookeeper zkServer = null;
+  private static final ZkKeyBuilder KEY_BUILDER = new ZkKeyBuilder("test");
+  private ZkConnection zkConnection = null;
+  private ZkClient zkClient = null;
+  private static final int SESSION_TIMEOUT_MS = 20000;
+  private static final int CONNECTION_TIMEOUT_MS = 10000;
+
+  @BeforeClass
+  public static void setup() throws InterruptedException {
+    zkServer = new EmbeddedZookeeper();
+    zkServer.setup();
+  }
+
+  @Before
+  public void testSetup() {
+    try {
+      zkClient = new ZkClient(
+          new ZkConnection("localhost:" + zkServer.getPort(), SESSION_TIMEOUT_MS),
+          CONNECTION_TIMEOUT_MS);
+    } catch (Exception e) {
+      Assert.fail("Client connection setup failed. Aborting tests..");
+    }
+    try {
+      zkClient.createPersistent(KEY_BUILDER.getProcessorsPath(), true);
+    } catch (ZkNodeExistsException e) {
+      // Do nothing
+    }
+  }
+
+
+  @After
+  public void testTeardown() {
+    zkClient.close();
+  }
+
+  @AfterClass
+  public static void teardown() {
+    zkServer.teardown();
+  }
+
+  @Test
+  public void testRegisterProcessorId() {
+    ZkUtils utils = new ZkUtils(
+        KEY_BUILDER,
+        zkClient,
+        SESSION_TIMEOUT_MS);
+    utils.connect();
+    String assignedPath = utils.registerProcessorAndGetId("0.0.0.0");
+    Assert.assertTrue(assignedPath.startsWith(KEY_BUILDER.getProcessorsPath()));
+
+    // Calling registerProcessorId again should return the same ephemeralPath as long as the session is valid
+    Assert.assertTrue(utils.registerProcessorAndGetId("0.0.0.0").equals(assignedPath));
+
+    utils.close();
+  }
+
+  @Test
+  public void testGetActiveProcessors() {
+    ZkUtils utils = new ZkUtils(
+        KEY_BUILDER,
+        zkClient,
+        SESSION_TIMEOUT_MS);
+    utils.connect();
+
+    Assert.assertEquals(0, utils.getSortedActiveProcessors().size());
+    utils.registerProcessorAndGetId("processorData");
+
+    Assert.assertEquals(1, utils.getSortedActiveProcessors().size());
+
+    utils.close();
+  }
+
+}


[04/14] samza git commit: SAMZA-1073: top-level fluent API `

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/test/java/org/apache/samza/example/TestFluentStreamTasks.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/TestFluentStreamTasks.java b/samza-operator/src/test/java/org/apache/samza/example/TestFluentStreamTasks.java
new file mode 100644
index 0000000..5f659ba
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/TestFluentStreamTasks.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.lang.reflect.Field;
+import org.apache.samza.Partition;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.impl.OperatorGraph;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.task.StreamOperatorTask;
+import org.apache.samza.task.TaskContext;
+import org.junit.Test;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+
+/**
+ * Unit test for {@link StreamOperatorTask}
+ */
+public class TestFluentStreamTasks {
+
+  private final Set<SystemStreamPartition> inputPartitions = new HashSet<SystemStreamPartition>() { {
+      for (int i = 0; i < 4; i++) {
+        this.add(new SystemStreamPartition("my-system", String.format("my-topic%d", i), new Partition(i)));
+      }
+    } };
+
+  @Test
+  public void testUserTask() throws Exception {
+    Config mockConfig = mock(Config.class);
+    TaskContext mockContext = mock(TaskContext.class);
+    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
+    WindowGraph userTask = new WindowGraph(this.inputPartitions);
+    StreamOperatorTask adaptorTask = new StreamOperatorTask(userTask);
+    Field pipelineMapFld = StreamOperatorTask.class.getDeclaredField("operatorGraph");
+    pipelineMapFld.setAccessible(true);
+    OperatorGraph opGraph = (OperatorGraph) pipelineMapFld.get(adaptorTask);
+
+    adaptorTask.init(mockConfig, mockContext);
+    this.inputPartitions.forEach(partition -> {
+        assertNotNull(opGraph.get(partition.getSystemStream()));
+      });
+  }
+
+  @Test
+  public void testSplitTask() throws Exception {
+    Config mockConfig = mock(Config.class);
+    TaskContext mockContext = mock(TaskContext.class);
+    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
+    BroadcastGraph splitTask = new BroadcastGraph(this.inputPartitions);
+    StreamOperatorTask adaptorTask = new StreamOperatorTask(splitTask);
+    Field pipelineMapFld = StreamOperatorTask.class.getDeclaredField("operatorGraph");
+    pipelineMapFld.setAccessible(true);
+    OperatorGraph opGraph = (OperatorGraph) pipelineMapFld.get(adaptorTask);
+
+    adaptorTask.init(mockConfig, mockContext);
+    this.inputPartitions.forEach(partition -> {
+        assertNotNull(opGraph.get(partition.getSystemStream()));
+      });
+  }
+
+  @Test
+  public void testJoinTask() throws Exception {
+    Config mockConfig = mock(Config.class);
+    TaskContext mockContext = mock(TaskContext.class);
+    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
+    JoinGraph joinTask = new JoinGraph(this.inputPartitions);
+    StreamOperatorTask adaptorTask = new StreamOperatorTask(joinTask);
+    Field pipelineMapFld = StreamOperatorTask.class.getDeclaredField("operatorGraph");
+    pipelineMapFld.setAccessible(true);
+    OperatorGraph opGraph = (OperatorGraph) pipelineMapFld.get(adaptorTask);
+
+    adaptorTask.init(mockConfig, mockContext);
+    this.inputPartitions.forEach(partition -> {
+        assertNotNull(opGraph.get(partition.getSystemStream()));
+      });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/test/java/org/apache/samza/example/WindowGraph.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/WindowGraph.java b/samza-operator/src/test/java/org/apache/samza/example/WindowGraph.java
new file mode 100644
index 0000000..a365411
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/WindowGraph.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.config.Config;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamGraphFactory;
+import org.apache.samza.operators.StreamGraphImpl;
+import org.apache.samza.operators.StreamSpec;
+import org.apache.samza.operators.data.InputMessageEnvelope;
+import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.data.Offset;
+import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.time.Duration;
+import java.util.function.BiFunction;
+import java.util.Properties;
+import java.util.Set;
+
+
+/**
+ * Example implementation of a simple user-defined tasks w/ window operators
+ *
+ */
+public class WindowGraph implements StreamGraphFactory {
+  class MessageType {
+    String field1;
+    String field2;
+  }
+
+  private final Set<SystemStreamPartition> inputs;
+
+  WindowGraph(Set<SystemStreamPartition> inputs) {
+    this.inputs = inputs;
+  }
+
+  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
+
+    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
+      super(key, data, offset, partition);
+    }
+  }
+
+  @Override
+  public StreamGraph create(Config config) {
+    StreamGraphImpl graph = new StreamGraphImpl();
+    BiFunction<JsonMessageEnvelope, Integer, Integer> maxAggregator = (m, c) -> c + 1;
+    inputs.forEach(source -> graph.<Object, Object, InputMessageEnvelope>createInStream(new StreamSpec() {
+      @Override public SystemStream getSystemStream() {
+        return source.getSystemStream();
+      }
+
+      @Override public Properties getProperties() {
+        return null;
+      }
+    }, null, null).
+        map(m1 -> new JsonMessageEnvelope(this.myMessageKeyFunction(m1), (MessageType) m1.getMessage(), m1.getOffset(),
+            m1.getSystemStreamPartition())).window(Windows.tumblingWindow(Duration.ofMillis(200), maxAggregator)));
+
+    return graph;
+  }
+
+  String myMessageKeyFunction(MessageEnvelope<Object, Object> m) {
+    return m.getKey().toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/test/java/org/apache/samza/operators/BroadcastTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/BroadcastTask.java b/samza-operator/src/test/java/org/apache/samza/operators/BroadcastTask.java
deleted file mode 100644
index 663d98c..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/BroadcastTask.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.operators;
-
-
-import org.apache.samza.operators.data.IncomingSystemMessageEnvelope;
-import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
-import org.apache.samza.operators.data.Offset;
-import org.apache.samza.operators.triggers.Triggers;
-import org.apache.samza.operators.windows.Windows;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.time.Duration;
-import java.util.Map;
-import java.util.function.BiFunction;
-
-
-/**
- * Example implementation of split stream tasks
- *
- */
-public class BroadcastTask implements StreamOperatorTask {
-  class MessageType {
-    String field1;
-    String field2;
-    String field3;
-    String field4;
-    String parKey;
-    private long timestamp;
-
-    public long getTimestamp() {
-      return this.timestamp;
-    }
-  }
-
-  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
-    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
-      super(key, data, offset, partition);
-    }
-  }
-
-  @Override
-  public void transform(Map<SystemStreamPartition, MessageStream<IncomingSystemMessageEnvelope>> messageStreams) {
-    BiFunction<JsonMessageEnvelope, Integer, Integer> sumAggregator = (m, c) -> c + 1;
-    messageStreams.values().forEach(entry -> {
-        MessageStream<JsonMessageEnvelope> inputStream = entry.map(this::getInputMessage);
-
-        inputStream.filter(this::myFilter1)
-          .window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
-            .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
-
-        inputStream.filter(this::myFilter1)
-          .window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
-            .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
-
-        inputStream.filter(this::myFilter1)
-          .window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
-            .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
-      });
-  }
-
-  JsonMessageEnvelope getInputMessage(IncomingSystemMessageEnvelope m1) {
-    return (JsonMessageEnvelope) m1.getMessage();
-  }
-
-  boolean myFilter1(JsonMessageEnvelope m1) {
-    // Do user defined processing here
-    return m1.getMessage().parKey.equals("key1");
-  }
-
-  boolean myFilter2(JsonMessageEnvelope m1) {
-    // Do user defined processing here
-    return m1.getMessage().parKey.equals("key2");
-  }
-
-  boolean myFilter3(JsonMessageEnvelope m1) {
-    return m1.getMessage().parKey.equals("key3");
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/test/java/org/apache/samza/operators/JoinTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/JoinTask.java b/samza-operator/src/test/java/org/apache/samza/operators/JoinTask.java
deleted file mode 100644
index 1b10609..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/JoinTask.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.operators;
-
-import org.apache.samza.operators.data.IncomingSystemMessageEnvelope;
-import org.apache.samza.operators.data.Offset;
-import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-
-/**
- * Example implementation of unique key-based stream-stream join tasks
- *
- */
-public class JoinTask implements StreamOperatorTask {
-  class MessageType {
-    String joinKey;
-    List<String> joinFields = new ArrayList<>();
-  }
-
-  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
-    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
-      super(key, data, offset, partition);
-    }
-  }
-
-  MessageStream<JsonMessageEnvelope> joinOutput = null;
-
-  @Override
-  public void transform(Map<SystemStreamPartition, MessageStream<IncomingSystemMessageEnvelope>> messageStreams) {
-    messageStreams.values().forEach(messageStream -> {
-        MessageStream<JsonMessageEnvelope> newSource = messageStream.map(this::getInputMessage);
-        if (joinOutput == null) {
-          joinOutput = newSource;
-        } else {
-          joinOutput = joinOutput.join(newSource, (m1, m2) -> this.myJoinResult(m1, m2));
-        }
-      });
-  }
-
-  private JsonMessageEnvelope getInputMessage(IncomingSystemMessageEnvelope ism) {
-    return new JsonMessageEnvelope(
-        ((MessageType) ism.getMessage()).joinKey,
-        (MessageType) ism.getMessage(),
-        ism.getOffset(),
-        ism.getSystemStreamPartition());
-  }
-
-  JsonMessageEnvelope myJoinResult(JsonMessageEnvelope m1, JsonMessageEnvelope m2) {
-    MessageType newJoinMsg = new MessageType();
-    newJoinMsg.joinKey = m1.getKey();
-    newJoinMsg.joinFields.addAll(m1.getMessage().joinFields);
-    newJoinMsg.joinFields.addAll(m2.getMessage().joinFields);
-    return new JsonMessageEnvelope(m1.getMessage().joinKey, newJoinMsg, null, null);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamAdaptorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamAdaptorTask.java b/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamAdaptorTask.java
deleted file mode 100644
index 61bb32a..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamAdaptorTask.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators;
-
-import org.apache.samza.Partition;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.impl.OperatorImpl;
-import org.apache.samza.system.SystemStreamPartition;
-import org.apache.samza.task.TaskContext;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-import java.lang.reflect.Field;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-
-public class TestFluentStreamAdaptorTask {
-  Field userTaskField = null;
-  Field operatorChainsField = null;
-
-  @Before
-  public void prep() throws NoSuchFieldException {
-    userTaskField = StreamOperatorAdaptorTask.class.getDeclaredField("userTask");
-    operatorChainsField = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
-    userTaskField.setAccessible(true);
-    operatorChainsField.setAccessible(true);
-  }
-
-  @Test
-  public void testConstructor() throws IllegalAccessException {
-    StreamOperatorTask userTask = mock(StreamOperatorTask.class);
-    StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(userTask);
-    StreamOperatorTask taskMemberVar = (StreamOperatorTask) userTaskField.get(adaptorTask);
-    Map<SystemStreamPartition, OperatorImpl> chainsMap = (Map<SystemStreamPartition, OperatorImpl>) operatorChainsField.get(adaptorTask);
-    assertEquals(taskMemberVar, userTask);
-    assertTrue(chainsMap.isEmpty());
-  }
-
-  @Test
-  public void testInit() throws Exception {
-    StreamOperatorTask userTask = mock(StreamOperatorTask.class);
-    StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(userTask);
-    Config mockConfig = mock(Config.class);
-    TaskContext mockContext = mock(TaskContext.class);
-    Set<SystemStreamPartition> testInputs = new HashSet() { {
-        this.add(new SystemStreamPartition("test-sys", "test-strm", new Partition(0)));
-        this.add(new SystemStreamPartition("test-sys", "test-strm", new Partition(1)));
-      } };
-    when(mockContext.getSystemStreamPartitions()).thenReturn(testInputs);
-    adaptorTask.init(mockConfig, mockContext);
-    verify(userTask, times(1)).transform(Mockito.anyMap());
-    Map<SystemStreamPartition, OperatorImpl> chainsMap = (Map<SystemStreamPartition, OperatorImpl>) operatorChainsField.get(adaptorTask);
-    assertTrue(chainsMap.size() == 2);
-    assertTrue(chainsMap.containsKey(testInputs.toArray()[0]));
-    assertTrue(chainsMap.containsKey(testInputs.toArray()[1]));
-  }
-
-  // TODO: window and process methods to be added after implementation of ChainedOperators.create()
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamTasks.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamTasks.java b/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamTasks.java
deleted file mode 100644
index d804bf8..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamTasks.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators;
-
-import org.apache.samza.Partition;
-import org.apache.samza.config.Config;
-
-import org.apache.samza.operators.impl.OperatorImpl;
-import org.apache.samza.system.SystemStreamPartition;
-import org.apache.samza.task.TaskContext;
-import org.junit.Test;
-
-import java.lang.reflect.Field;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-
-/**
- * Unit test for {@link StreamOperatorTask}
- */
-public class TestFluentStreamTasks {
-
-  private final WindowTask userTask = new WindowTask();
-
-  private final BroadcastTask splitTask = new BroadcastTask();
-
-  private final JoinTask joinTask = new JoinTask();
-
-  private final Set<SystemStreamPartition> inputPartitions = new HashSet<SystemStreamPartition>() { {
-      for (int i = 0; i < 4; i++) {
-        this.add(new SystemStreamPartition("my-system", "my-topic1", new Partition(i)));
-      }
-    } };
-
-  @Test
-  public void testUserTask() throws Exception {
-    Config mockConfig = mock(Config.class);
-    TaskContext mockContext = mock(TaskContext.class);
-    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
-    StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(this.userTask);
-    Field pipelineMapFld = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
-    pipelineMapFld.setAccessible(true);
-    Map<SystemStreamPartition, OperatorImpl> pipelineMap =
-        (Map<SystemStreamPartition, OperatorImpl>) pipelineMapFld.get(adaptorTask);
-
-    adaptorTask.init(mockConfig, mockContext);
-    assertEquals(pipelineMap.size(), 4);
-    this.inputPartitions.forEach(partition -> {
-        assertNotNull(pipelineMap.get(partition));
-      });
-  }
-
-  @Test
-  public void testSplitTask() throws Exception {
-    Config mockConfig = mock(Config.class);
-    TaskContext mockContext = mock(TaskContext.class);
-    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
-    StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(this.splitTask);
-    Field pipelineMapFld = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
-    pipelineMapFld.setAccessible(true);
-    Map<SystemStreamPartition, OperatorImpl> pipelineMap =
-        (Map<SystemStreamPartition, OperatorImpl>) pipelineMapFld.get(adaptorTask);
-
-    adaptorTask.init(mockConfig, mockContext);
-    assertEquals(pipelineMap.size(), 4);
-    this.inputPartitions.forEach(partition -> {
-        assertNotNull(pipelineMap.get(partition));
-      });
-  }
-
-  @Test
-  public void testJoinTask() throws Exception {
-    Config mockConfig = mock(Config.class);
-    TaskContext mockContext = mock(TaskContext.class);
-    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
-    StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(this.joinTask);
-    Field pipelineMapFld = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
-    pipelineMapFld.setAccessible(true);
-    Map<SystemStreamPartition, OperatorImpl> pipelineMap =
-        (Map<SystemStreamPartition, OperatorImpl>) pipelineMapFld.get(adaptorTask);
-
-    adaptorTask.init(mockConfig, mockContext);
-    assertEquals(pipelineMap.size(), 4);
-    this.inputPartitions.forEach(partition -> {
-        assertNotNull(pipelineMap.get(partition));
-      });
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
index 5991e2f..d5607d8 100644
--- a/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
+++ b/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
@@ -21,6 +21,7 @@ package org.apache.samza.operators;
 import org.apache.samza.operators.functions.FilterFunction;
 import org.apache.samza.operators.functions.FlatMapFunction;
 import org.apache.samza.operators.functions.JoinFunction;
+import org.apache.samza.operators.functions.KeyValueJoinFunction;
 import org.apache.samza.operators.functions.MapFunction;
 import org.apache.samza.operators.functions.SinkFunction;
 import org.apache.samza.operators.spec.OperatorSpec;
@@ -29,6 +30,7 @@ import org.apache.samza.operators.spec.SinkOperatorSpec;
 import org.apache.samza.operators.spec.StreamOperatorSpec;
 import org.apache.samza.system.OutgoingMessageEnvelope;
 import org.apache.samza.system.SystemStream;
+import org.apache.samza.task.MessageCollector;
 import org.apache.samza.task.TaskCoordinator;
 import org.junit.Test;
 
@@ -46,17 +48,19 @@ import static org.mockito.Mockito.when;
 
 public class TestMessageStreamImpl {
 
+  private StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+
   @Test
   public void testMap() {
-    MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>();
-    MapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> xMap =
-        m -> new TestOutputMessageEnvelope(m.getKey(), m.getMessage().getValue().length() + 1);
+    MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>(mockGraph);
+    MapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> xMap = (TestMessageEnvelope m)  ->
+        new TestOutputMessageEnvelope(m.getKey(), m.getMessage().getValue().length() + 1);
     MessageStream<TestOutputMessageEnvelope> outputStream = inputStream.map(xMap);
     Collection<OperatorSpec> subs = inputStream.getRegisteredOperatorSpecs();
     assertEquals(subs.size(), 1);
     OperatorSpec<TestOutputMessageEnvelope> mapOp = subs.iterator().next();
     assertTrue(mapOp instanceof StreamOperatorSpec);
-    assertEquals(mapOp.getOutputStream(), outputStream);
+    assertEquals(mapOp.getNextStream(), outputStream);
     // assert that the transformation function is what we defined above
     TestMessageEnvelope xTestMsg = mock(TestMessageEnvelope.class);
     TestMessageEnvelope.MessageType mockInnerTestMessage = mock(TestMessageEnvelope.MessageType.class);
@@ -73,33 +77,33 @@ public class TestMessageStreamImpl {
 
   @Test
   public void testFlatMap() {
-    MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>();
+    MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>(mockGraph);
     Set<TestOutputMessageEnvelope> flatOuts = new HashSet<TestOutputMessageEnvelope>() { {
         this.add(mock(TestOutputMessageEnvelope.class));
         this.add(mock(TestOutputMessageEnvelope.class));
         this.add(mock(TestOutputMessageEnvelope.class));
       } };
-    FlatMapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> xFlatMap = m -> flatOuts;
+    FlatMapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> xFlatMap = (TestMessageEnvelope message) -> flatOuts;
     MessageStream<TestOutputMessageEnvelope> outputStream = inputStream.flatMap(xFlatMap);
     Collection<OperatorSpec> subs = inputStream.getRegisteredOperatorSpecs();
     assertEquals(subs.size(), 1);
     OperatorSpec<TestOutputMessageEnvelope> flatMapOp = subs.iterator().next();
     assertTrue(flatMapOp instanceof StreamOperatorSpec);
-    assertEquals(flatMapOp.getOutputStream(), outputStream);
+    assertEquals(flatMapOp.getNextStream(), outputStream);
     // assert that the transformation function is what we defined above
     assertEquals(((StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope>) flatMapOp).getTransformFn(), xFlatMap);
   }
 
   @Test
   public void testFilter() {
-    MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>();
-    FilterFunction<TestMessageEnvelope> xFilter = m -> m.getMessage().getEventTime() > 123456L;
+    MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>(mockGraph);
+    FilterFunction<TestMessageEnvelope> xFilter = (TestMessageEnvelope m) -> m.getMessage().getEventTime() > 123456L;
     MessageStream<TestMessageEnvelope> outputStream = inputStream.filter(xFilter);
     Collection<OperatorSpec> subs = inputStream.getRegisteredOperatorSpecs();
     assertEquals(subs.size(), 1);
     OperatorSpec<TestMessageEnvelope> filterOp = subs.iterator().next();
     assertTrue(filterOp instanceof StreamOperatorSpec);
-    assertEquals(filterOp.getOutputStream(), outputStream);
+    assertEquals(filterOp.getNextStream(), outputStream);
     // assert that the transformation function is what we defined above
     FlatMapFunction<TestMessageEnvelope, TestMessageEnvelope> txfmFn = ((StreamOperatorSpec<TestMessageEnvelope, TestMessageEnvelope>) filterOp).getTransformFn();
     TestMessageEnvelope mockMsg = mock(TestMessageEnvelope.class);
@@ -117,8 +121,8 @@ public class TestMessageStreamImpl {
 
   @Test
   public void testSink() {
-    MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>();
-    SinkFunction<TestMessageEnvelope> xSink = (m, mc, tc) -> {
+    MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>(mockGraph);
+    SinkFunction<TestMessageEnvelope> xSink = (TestMessageEnvelope m, MessageCollector mc, TaskCoordinator tc) -> {
       mc.send(new OutgoingMessageEnvelope(new SystemStream("test-sys", "test-stream"), m.getMessage()));
       tc.commit(TaskCoordinator.RequestScope.CURRENT_TASK);
     };
@@ -128,26 +132,28 @@ public class TestMessageStreamImpl {
     OperatorSpec<TestMessageEnvelope> sinkOp = subs.iterator().next();
     assertTrue(sinkOp instanceof SinkOperatorSpec);
     assertEquals(((SinkOperatorSpec) sinkOp).getSinkFn(), xSink);
-    assertNull(((SinkOperatorSpec) sinkOp).getOutputStream());
+    assertNull(((SinkOperatorSpec) sinkOp).getNextStream());
   }
 
   @Test
   public void testJoin() {
-    MessageStreamImpl<TestMessageEnvelope> source1 = new MessageStreamImpl<>();
-    MessageStreamImpl<TestMessageEnvelope> source2 = new MessageStreamImpl<>();
-    JoinFunction<TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope> joiner =
-        (m1, m2) -> new TestOutputMessageEnvelope(m1.getKey(), m1.getMessage().getValue().length() + m2.getMessage().getValue().length());
+    MessageStreamImpl<TestMessageEnvelope> source1 = new MessageStreamImpl<>(mockGraph);
+    MessageStreamImpl<TestMessageEnvelope> source2 = new MessageStreamImpl<>(mockGraph);
+    JoinFunction<String, TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope> joiner =
+        (KeyValueJoinFunction<String, TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope>)
+            (m1, m2) -> new TestOutputMessageEnvelope(m1.getKey(), m1.getMessage().getValue().length() + m2.getMessage().getValue().length());
+
     MessageStream<TestOutputMessageEnvelope> joinOutput = source1.join(source2, joiner);
     Collection<OperatorSpec> subs = source1.getRegisteredOperatorSpecs();
     assertEquals(subs.size(), 1);
     OperatorSpec<TestMessageEnvelope> joinOp1 = subs.iterator().next();
     assertTrue(joinOp1 instanceof PartialJoinOperatorSpec);
-    assertEquals(((PartialJoinOperatorSpec) joinOp1).getOutputStream(), joinOutput);
+    assertEquals(((PartialJoinOperatorSpec) joinOp1).getNextStream(), joinOutput);
     subs = source2.getRegisteredOperatorSpecs();
     assertEquals(subs.size(), 1);
     OperatorSpec<TestMessageEnvelope> joinOp2 = subs.iterator().next();
     assertTrue(joinOp2 instanceof PartialJoinOperatorSpec);
-    assertEquals(((PartialJoinOperatorSpec) joinOp2).getOutputStream(), joinOutput);
+    assertEquals(((PartialJoinOperatorSpec) joinOp2).getNextStream(), joinOutput);
     TestMessageEnvelope joinMsg1 = new TestMessageEnvelope("test-join-1", "join-msg-001", 11111L);
     TestMessageEnvelope joinMsg2 = new TestMessageEnvelope("test-join-2", "join-msg-002", 22222L);
     TestOutputMessageEnvelope xOut = (TestOutputMessageEnvelope) ((PartialJoinOperatorSpec) joinOp1).getTransformFn().apply(joinMsg1, joinMsg2);
@@ -160,10 +166,10 @@ public class TestMessageStreamImpl {
 
   @Test
   public void testMerge() {
-    MessageStream<TestMessageEnvelope> merge1 = new MessageStreamImpl<>();
+    MessageStream<TestMessageEnvelope> merge1 = new MessageStreamImpl<>(mockGraph);
     Collection<MessageStream<TestMessageEnvelope>> others = new ArrayList<MessageStream<TestMessageEnvelope>>() { {
-        this.add(new MessageStreamImpl<>());
-        this.add(new MessageStreamImpl<>());
+        this.add(new MessageStreamImpl<>(mockGraph));
+        this.add(new MessageStreamImpl<>(mockGraph));
       } };
     MessageStream<TestMessageEnvelope> mergeOutput = merge1.merge(others);
     validateMergeOperator(merge1, mergeOutput);
@@ -176,7 +182,7 @@ public class TestMessageStreamImpl {
     assertEquals(subs.size(), 1);
     OperatorSpec<TestMessageEnvelope> mergeOp = subs.iterator().next();
     assertTrue(mergeOp instanceof StreamOperatorSpec);
-    assertEquals(((StreamOperatorSpec) mergeOp).getOutputStream(), mergeOutput);
+    assertEquals(((StreamOperatorSpec) mergeOp).getNextStream(), mergeOutput);
     TestMessageEnvelope mockMsg = mock(TestMessageEnvelope.class);
     Collection<TestMessageEnvelope> outputs = ((StreamOperatorSpec<TestMessageEnvelope, TestMessageEnvelope>) mergeOp).getTransformFn().apply(mockMsg);
     assertEquals(outputs.size(), 1);

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

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/test/java/org/apache/samza/operators/WindowTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/WindowTask.java b/samza-operator/src/test/java/org/apache/samza/operators/WindowTask.java
deleted file mode 100644
index e176063..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/WindowTask.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.operators;
-
-import org.apache.samza.operators.data.IncomingSystemMessageEnvelope;
-import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
-import org.apache.samza.operators.data.MessageEnvelope;
-import org.apache.samza.operators.data.Offset;
-import org.apache.samza.operators.windows.Windows;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.time.Duration;
-import java.util.Map;
-import java.util.function.BiFunction;
-
-
-/**
- * Example implementation of a simple user-defined tasks w/ window operators
- *
- */
-public class WindowTask implements StreamOperatorTask {
-  class MessageType {
-    String field1;
-    String field2;
-  }
-
-  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
-
-    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
-      super(key, data, offset, partition);
-    }
-  }
-
-  @Override public void transform(Map<SystemStreamPartition, MessageStream<IncomingSystemMessageEnvelope>> messageStreams) {
-    BiFunction<JsonMessageEnvelope, Integer, Integer> maxAggregator = (m, c) -> c + 1;
-    messageStreams.values().forEach(source ->
-        source.map(m1 -> new JsonMessageEnvelope(this.myMessageKeyFunction(m1), (MessageType) m1.getMessage(), m1.getOffset(), m1.getSystemStreamPartition()))
-            .window(Windows.tumblingWindow(Duration.ofMillis(200), maxAggregator))
-    );
-  }
-
-  String myMessageKeyFunction(MessageEnvelope<Object, Object> m) {
-    return m.getKey().toString();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java
index bae98e3..ec63d41 100644
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java
@@ -18,17 +18,26 @@
  */
 package org.apache.samza.operators.impl;
 
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import org.apache.samza.config.Config;
 import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.StreamGraphImpl;
 import org.apache.samza.operators.TestMessageEnvelope;
+import org.apache.samza.operators.TestMessageStreamImplUtil;
 import org.apache.samza.operators.TestOutputMessageEnvelope;
 import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.functions.KeyValueJoinFunction;
+import org.apache.samza.operators.functions.PartialJoinFunction;
 import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.operators.spec.OperatorSpec;
 import org.apache.samza.operators.spec.WindowOperatorSpec;
 import org.apache.samza.operators.spec.PartialJoinOperatorSpec;
 import org.apache.samza.operators.spec.SinkOperatorSpec;
 import org.apache.samza.operators.spec.StreamOperatorSpec;
 import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.operators.windows.internal.WindowInternal;
 import org.apache.samza.task.TaskContext;
 import org.junit.Before;
 import org.junit.Test;
@@ -38,7 +47,6 @@ import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.Set;
-import java.util.function.BiFunction;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotSame;
@@ -49,25 +57,46 @@ import static org.mockito.Mockito.when;
 
 public class TestOperatorImpls {
   Field nextOperatorsField = null;
+  Method createOpMethod = null;
+  Method createOpsMethod = null;
 
   @Before
-  public void prep() throws NoSuchFieldException {
+  public void prep() throws NoSuchFieldException, NoSuchMethodException {
     nextOperatorsField = OperatorImpl.class.getDeclaredField("nextOperators");
     nextOperatorsField.setAccessible(true);
+
+    createOpMethod = OperatorGraph.class.getDeclaredMethod("createOperatorImpl", MessageStreamImpl.class,
+        OperatorSpec.class, Config.class, TaskContext.class);
+    createOpMethod.setAccessible(true);
+
+    createOpsMethod = OperatorGraph.class.getDeclaredMethod("createOperatorImpls", MessageStreamImpl.class, Config.class, TaskContext.class);
+    createOpsMethod.setAccessible(true);
   }
-  
+
   @Test
-  public void testCreateOperator() throws NoSuchFieldException, IllegalAccessException {
+  public void testCreateOperator() throws NoSuchFieldException, IllegalAccessException, InvocationTargetException {
     // get window operator
     WindowOperatorSpec mockWnd = mock(WindowOperatorSpec.class);
-    OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope> opImpl = OperatorImpls.createOperatorImpl(mockWnd);
+    WindowInternal<TestMessageEnvelope, String, Integer> windowInternal = new WindowInternal<>(null, null, null, null);
+    when(mockWnd.getWindow()).thenReturn(windowInternal);
+    MessageStreamImpl<TestMessageEnvelope> mockStream = mock(MessageStreamImpl.class);
+    Config mockConfig = mock(Config.class);
+    TaskContext mockContext = mock(TaskContext.class);
+
+    OperatorGraph opGraph = new OperatorGraph();
+    OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope> opImpl = (OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope>)
+        createOpMethod.invoke(opGraph, mockStream, mockWnd, mockConfig, mockContext);
     assertTrue(opImpl instanceof WindowOperatorImpl);
+    Field wndInternalField = WindowOperatorImpl.class.getDeclaredField("window");
+    wndInternalField.setAccessible(true);
+    WindowInternal wndInternal = (WindowInternal) wndInternalField.get(opImpl);
+    assertEquals(wndInternal, windowInternal);
 
     // get simple operator
     StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> mockSimpleOp = mock(StreamOperatorSpec.class);
     FlatMapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> mockTxfmFn = mock(FlatMapFunction.class);
     when(mockSimpleOp.getTransformFn()).thenReturn(mockTxfmFn);
-    opImpl = OperatorImpls.createOperatorImpl(mockSimpleOp);
+    opImpl = (OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope>) createOpMethod.invoke(opGraph, mockStream, mockSimpleOp, mockConfig, mockContext);
     assertTrue(opImpl instanceof StreamOperatorImpl);
     Field txfmFnField = StreamOperatorImpl.class.getDeclaredField("transformFn");
     txfmFnField.setAccessible(true);
@@ -77,7 +106,7 @@ public class TestOperatorImpls {
     SinkFunction<TestMessageEnvelope> sinkFn = (m, mc, tc) -> { };
     SinkOperatorSpec<TestMessageEnvelope> sinkOp = mock(SinkOperatorSpec.class);
     when(sinkOp.getSinkFn()).thenReturn(sinkFn);
-    opImpl = OperatorImpls.createOperatorImpl(sinkOp);
+    opImpl = (OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope>) createOpMethod.invoke(opGraph, mockStream, sinkOp, mockConfig, mockContext);
     assertTrue(opImpl instanceof SinkOperatorImpl);
     Field sinkFnField = SinkOperatorImpl.class.getDeclaredField("sinkFn");
     sinkFnField.setAccessible(true);
@@ -86,28 +115,33 @@ public class TestOperatorImpls {
     // get join operator
     PartialJoinOperatorSpec<TestMessageEnvelope, String, TestMessageEnvelope, TestOutputMessageEnvelope> joinOp = mock(PartialJoinOperatorSpec.class);
     TestOutputMessageEnvelope mockOutput = mock(TestOutputMessageEnvelope.class);
-    BiFunction<TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope> joinFn = (m1, m2) -> mockOutput;
+    PartialJoinFunction<String, TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope> joinFn = mock(PartialJoinFunction.class);
     when(joinOp.getTransformFn()).thenReturn(joinFn);
-    opImpl = OperatorImpls.createOperatorImpl(joinOp);
+    opImpl = (OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope>) createOpMethod.invoke(opGraph, mockStream, joinOp, mockConfig, mockContext);
     assertTrue(opImpl instanceof PartialJoinOperatorImpl);
   }
 
   @Test
-  public void testEmptyChain() {
+  public void testEmptyChain() throws InvocationTargetException, IllegalAccessException {
     // test creation of empty chain
-    MessageStreamImpl<TestMessageEnvelope> testStream = new MessageStreamImpl<>();
+    MessageStreamImpl<TestMessageEnvelope> testStream = mock(MessageStreamImpl.class);
     TaskContext mockContext = mock(TaskContext.class);
-    RootOperatorImpl operatorChain = OperatorImpls.createOperatorImpls(testStream, mockContext);
+    Config mockConfig = mock(Config.class);
+    OperatorGraph opGraph = new OperatorGraph();
+    RootOperatorImpl operatorChain = (RootOperatorImpl) createOpsMethod.invoke(opGraph, testStream, mockConfig, mockContext);
     assertTrue(operatorChain != null);
   }
 
   @Test
-  public void testLinearChain() throws IllegalAccessException {
+  public void testLinearChain() throws IllegalAccessException, InvocationTargetException {
     // test creation of linear chain
-    MessageStreamImpl<TestMessageEnvelope> testInput = new MessageStreamImpl<>();
+    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    MessageStreamImpl<TestMessageEnvelope> testInput = TestMessageStreamImplUtil.<TestMessageEnvelope>getMessageStreamImpl(mockGraph);
     TaskContext mockContext = mock(TaskContext.class);
-    testInput.map(m -> m).window(Windows.tumblingWindow(Duration.ofMillis(1000)));
-    RootOperatorImpl operatorChain = OperatorImpls.createOperatorImpls(testInput, mockContext);
+    Config mockConfig = mock(Config.class);
+    testInput.map(m -> m).window(Windows.keyedSessionWindow(TestMessageEnvelope::getKey, Duration.ofMinutes(10)));
+    OperatorGraph opGraph = new OperatorGraph();
+    RootOperatorImpl operatorChain = (RootOperatorImpl) createOpsMethod.invoke(opGraph, testInput, mockConfig, mockContext);
     Set<OperatorImpl> subsSet = (Set<OperatorImpl>) nextOperatorsField.get(operatorChain);
     assertEquals(subsSet.size(), 1);
     OperatorImpl<TestMessageEnvelope, TestMessageEnvelope> firstOpImpl = subsSet.iterator().next();
@@ -119,13 +153,16 @@ public class TestOperatorImpls {
   }
 
   @Test
-  public void testBroadcastChain() throws IllegalAccessException {
+  public void testBroadcastChain() throws IllegalAccessException, InvocationTargetException {
     // test creation of broadcast chain
-    MessageStreamImpl<TestMessageEnvelope> testInput = new MessageStreamImpl<>();
+    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    MessageStreamImpl<TestMessageEnvelope> testInput = TestMessageStreamImplUtil.<TestMessageEnvelope>getMessageStreamImpl(mockGraph);
     TaskContext mockContext = mock(TaskContext.class);
+    Config mockConfig = mock(Config.class);
     testInput.filter(m -> m.getMessage().getEventTime() > 123456L).flatMap(m -> new ArrayList() { { this.add(m); this.add(m); } });
     testInput.filter(m -> m.getMessage().getEventTime() < 123456L).map(m -> m);
-    RootOperatorImpl operatorChain = OperatorImpls.createOperatorImpls(testInput, mockContext);
+    OperatorGraph opGraph = new OperatorGraph();
+    RootOperatorImpl operatorChain = (RootOperatorImpl) createOpsMethod.invoke(opGraph, testInput, mockConfig, mockContext);
     Set<OperatorImpl> subsSet = (Set<OperatorImpl>) nextOperatorsField.get(operatorChain);
     assertEquals(subsSet.size(), 2);
     Iterator<OperatorImpl> iter = subsSet.iterator();
@@ -146,18 +183,23 @@ public class TestOperatorImpls {
   }
 
   @Test
-  public void testJoinChain() throws IllegalAccessException {
+  public void testJoinChain() throws IllegalAccessException, InvocationTargetException {
     // test creation of join chain
-    MessageStreamImpl<TestMessageEnvelope> input1 = new MessageStreamImpl<>();
-    MessageStreamImpl<TestMessageEnvelope> input2 = new MessageStreamImpl<>();
+    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    MessageStreamImpl<TestMessageEnvelope> input1 = TestMessageStreamImplUtil.getMessageStreamImpl(mockGraph);
+    MessageStreamImpl<TestMessageEnvelope> input2 = TestMessageStreamImplUtil.getMessageStreamImpl(mockGraph);
     TaskContext mockContext = mock(TaskContext.class);
+    Config mockConfig = mock(Config.class);
     input1
-        .join(input2, (m1, m2) ->
-            new TestOutputMessageEnvelope(m1.getKey(), m1.getMessage().getValue().length() + m2.getMessage().getValue().length()))
+        .join(input2,
+            (KeyValueJoinFunction<String, TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope>) (m1, m2) ->
+                new TestOutputMessageEnvelope(m1.getKey(), m1.getMessage().getValue().length() + m2.getMessage().getValue().length())
+            )
         .map(m -> m);
+    OperatorGraph opGraph = new OperatorGraph();
     // now, we create chained operators from each input sources
-    RootOperatorImpl chain1 = OperatorImpls.createOperatorImpls(input1, mockContext);
-    RootOperatorImpl chain2 = OperatorImpls.createOperatorImpls(input2, mockContext);
+    RootOperatorImpl chain1 = (RootOperatorImpl) createOpsMethod.invoke(opGraph, input1, mockConfig, mockContext);
+    RootOperatorImpl chain2 = (RootOperatorImpl) createOpsMethod.invoke(opGraph, input2, mockConfig, mockContext);
     // check that those two chains will merge at map operator
     // first branch of the join
     Set<OperatorImpl> subsSet = (Set<OperatorImpl>) nextOperatorsField.get(chain1);

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
index ba5b6f8..ce9fdd2 100644
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
@@ -18,17 +18,16 @@
  */
 package org.apache.samza.operators.impl;
 
+import org.apache.samza.config.Config;
 import org.apache.samza.operators.TestOutputMessageEnvelope;
 import org.apache.samza.operators.functions.SinkFunction;
 import org.apache.samza.operators.spec.SinkOperatorSpec;
 import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
 import org.junit.Test;
 
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.*;
 
 
 public class TestSinkOperatorImpl {
@@ -38,7 +37,9 @@ public class TestSinkOperatorImpl {
     SinkOperatorSpec<TestOutputMessageEnvelope> sinkOp = mock(SinkOperatorSpec.class);
     SinkFunction<TestOutputMessageEnvelope> sinkFn = mock(SinkFunction.class);
     when(sinkOp.getSinkFn()).thenReturn(sinkFn);
-    SinkOperatorImpl<TestOutputMessageEnvelope> sinkImpl = new SinkOperatorImpl<>(sinkOp);
+    Config mockConfig = mock(Config.class);
+    TaskContext mockContext = mock(TaskContext.class);
+    SinkOperatorImpl<TestOutputMessageEnvelope> sinkImpl = new SinkOperatorImpl<>(sinkOp, mockConfig, mockContext);
     TestOutputMessageEnvelope mockMsg = mock(TestOutputMessageEnvelope.class);
     MessageCollector mockCollector = mock(MessageCollector.class);
     TaskCoordinator mockCoordinator = mock(TaskCoordinator.class);

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java
index 5a3840c..010a210 100644
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java
@@ -18,22 +18,20 @@
  */
 package org.apache.samza.operators.impl;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
 import org.apache.samza.operators.TestMessageEnvelope;
 import org.apache.samza.operators.TestOutputMessageEnvelope;
 import org.apache.samza.operators.functions.FlatMapFunction;
 import org.apache.samza.operators.spec.StreamOperatorSpec;
 import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
 import org.junit.Test;
 
-import java.util.ArrayList;
-import java.util.Collection;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.*;
 
 
 public class TestStreamOperatorImpl {
@@ -43,8 +41,10 @@ public class TestStreamOperatorImpl {
     StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> mockOp = mock(StreamOperatorSpec.class);
     FlatMapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> txfmFn = mock(FlatMapFunction.class);
     when(mockOp.getTransformFn()).thenReturn(txfmFn);
-
-    StreamOperatorImpl<TestMessageEnvelope, TestOutputMessageEnvelope> opImpl = spy(new StreamOperatorImpl<>(mockOp));
+    MessageStreamImpl<TestMessageEnvelope> mockInput = mock(MessageStreamImpl.class);
+    Config mockConfig = mock(Config.class);
+    TaskContext mockContext = mock(TaskContext.class);
+    StreamOperatorImpl<TestMessageEnvelope, TestOutputMessageEnvelope> opImpl = spy(new StreamOperatorImpl<>(mockOp, mockInput, mockConfig, mockContext));
     TestMessageEnvelope inMsg = mock(TestMessageEnvelope.class);
     TestOutputMessageEnvelope outMsg = mock(TestOutputMessageEnvelope.class);
     Collection<TestOutputMessageEnvelope> mockOutputs = new ArrayList() { {

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java b/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java
index ffe9df1..31257a4 100644
--- a/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java
+++ b/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java
@@ -18,14 +18,18 @@
  */
 package org.apache.samza.operators.spec;
 
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.StreamGraphImpl;
 import org.apache.samza.operators.TestMessageEnvelope;
+import org.apache.samza.operators.TestMessageStreamImplUtil;
 import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.functions.PartialJoinFunction;
 import org.apache.samza.operators.functions.SinkFunction;
-import org.apache.samza.operators.MessageStreamImpl;
 import org.apache.samza.operators.windows.internal.WindowInternal;
-import org.apache.samza.operators.windows.WindowKey;
 import org.apache.samza.operators.windows.WindowPane;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
 import org.junit.Test;
 
 import java.util.ArrayList;
@@ -42,19 +46,23 @@ public class TestOperatorSpecs {
   @Test
   public void testGetStreamOperator() {
     FlatMapFunction<MessageEnvelope, TestMessageEnvelope> transformFn = m -> new ArrayList<TestMessageEnvelope>() { {
-        this.add(new TestMessageEnvelope(m.getKey().toString(), m.getMessage().toString(), 12345L));
-      } };
-    StreamOperatorSpec<MessageEnvelope, TestMessageEnvelope> strmOp = OperatorSpecs.createStreamOperatorSpec(transformFn);
+          this.add(new TestMessageEnvelope(m.getKey().toString(), m.getMessage().toString(), 12345L));
+        } };
+    MessageStreamImpl<TestMessageEnvelope> mockOutput = mock(MessageStreamImpl.class);
+    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    StreamOperatorSpec<MessageEnvelope, TestMessageEnvelope> strmOp = OperatorSpecs.createStreamOperatorSpec(transformFn, mockGraph, mockOutput);
     assertEquals(strmOp.getTransformFn(), transformFn);
-    assertTrue(strmOp.getOutputStream() instanceof MessageStreamImpl);
+    assertEquals(strmOp.getNextStream(), mockOutput);
   }
 
   @Test
   public void testGetSinkOperator() {
-    SinkFunction<TestMessageEnvelope> sinkFn = (m, c, t) -> { };
-    SinkOperatorSpec<TestMessageEnvelope> sinkOp = OperatorSpecs.createSinkOperatorSpec(sinkFn);
+    SinkFunction<TestMessageEnvelope> sinkFn = (TestMessageEnvelope message, MessageCollector messageCollector,
+          TaskCoordinator taskCoordinator) -> { };
+    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    SinkOperatorSpec<TestMessageEnvelope> sinkOp = OperatorSpecs.createSinkOperatorSpec(sinkFn, mockGraph);
     assertEquals(sinkOp.getSinkFn(), sinkFn);
-    assertTrue(sinkOp.getOutputStream() == null);
+    assertTrue(sinkOp.getNextStream() == null);
   }
 
   @Test
@@ -65,8 +73,9 @@ public class TestOperatorSpecs {
     //instantiate a window using reflection
     WindowInternal window = new WindowInternal(null, aggregator, keyExtractor, null);
 
-    WindowOperatorSpec spec = OperatorSpecs.<TestMessageEnvelope, String, WindowKey<String>, Integer,
-        WindowPane<WindowKey<String>, Integer>>createWindowOperatorSpec(window);
+    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    MessageStreamImpl<WindowPane<String, Integer>> mockWndOut = mock(MessageStreamImpl.class);
+    WindowOperatorSpec spec = OperatorSpecs.<TestMessageEnvelope, String, Integer>createWindowOperatorSpec(window, mockGraph, mockWndOut);
     assertEquals(spec.getWindow(), window);
     assertEquals(spec.getWindow().getKeyExtractor(), keyExtractor);
     assertEquals(spec.getWindow().getFoldFunction(), aggregator);
@@ -74,13 +83,30 @@ public class TestOperatorSpecs {
 
   @Test
   public void testGetPartialJoinOperator() {
-    BiFunction<MessageEnvelope<Object, ?>, MessageEnvelope<Object, ?>, TestMessageEnvelope> merger =
-        (m1, m2) -> new TestMessageEnvelope(m1.getKey().toString(), m2.getMessage().toString(), System.nanoTime());
-    MessageStreamImpl<TestMessageEnvelope> joinOutput = new MessageStreamImpl<>();
+    PartialJoinFunction<Object, MessageEnvelope<Object, ?>, MessageEnvelope<Object, ?>, TestMessageEnvelope> merger =
+      new PartialJoinFunction<Object, MessageEnvelope<Object, ?>, MessageEnvelope<Object, ?>, TestMessageEnvelope>() {
+        @Override
+        public TestMessageEnvelope apply(MessageEnvelope<Object, ?> m1, MessageEnvelope<Object, ?> m2) {
+          return new TestMessageEnvelope(m1.getKey().toString(), m2.getMessage().toString(), System.nanoTime());
+        }
+
+        @Override
+        public Object getKey(MessageEnvelope<Object, ?> message) {
+          return message.getKey();
+        }
+
+        @Override
+        public Object getOtherKey(MessageEnvelope<Object, ?> message) {
+          return message.getKey();
+        }
+      };
+
+    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    MessageStreamImpl<TestMessageEnvelope> joinOutput = TestMessageStreamImplUtil.<TestMessageEnvelope>getMessageStreamImpl(mockGraph);
     PartialJoinOperatorSpec<MessageEnvelope<Object, ?>, Object, MessageEnvelope<Object, ?>, TestMessageEnvelope> partialJoin =
-        OperatorSpecs.createPartialJoinOperatorSpec(merger, joinOutput);
+        OperatorSpecs.createPartialJoinOperatorSpec(merger, mockGraph, joinOutput);
 
-    assertEquals(partialJoin.getOutputStream(), joinOutput);
+    assertEquals(partialJoin.getNextStream(), joinOutput);
     MessageEnvelope<Object, Object> m = mock(MessageEnvelope.class);
     MessageEnvelope<Object, Object> s = mock(MessageEnvelope.class);
     assertEquals(partialJoin.getTransformFn(), merger);
@@ -88,13 +114,14 @@ public class TestOperatorSpecs {
 
   @Test
   public void testGetMergeOperator() {
-    MessageStreamImpl<TestMessageEnvelope> output = new MessageStreamImpl<>();
-    StreamOperatorSpec<TestMessageEnvelope, TestMessageEnvelope> mergeOp = OperatorSpecs.createMergeOperatorSpec(output);
+    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    MessageStreamImpl<TestMessageEnvelope> output = TestMessageStreamImplUtil.<TestMessageEnvelope>getMessageStreamImpl(mockGraph);
+    StreamOperatorSpec<TestMessageEnvelope, TestMessageEnvelope> mergeOp = OperatorSpecs.createMergeOperatorSpec(mockGraph, output);
     Function<TestMessageEnvelope, Collection<TestMessageEnvelope>> mergeFn = t -> new ArrayList<TestMessageEnvelope>() { {
         this.add(t);
       } };
     TestMessageEnvelope t = mock(TestMessageEnvelope.class);
     assertEquals(mergeOp.getTransformFn().apply(t), mergeFn.apply(t));
-    assertEquals(mergeOp.getOutputStream(), output);
+    assertEquals(mergeOp.getNextStream(), output);
   }
 }


[08/14] samza git commit: SAMZA-1087: Schedule after debounce time

Posted by ni...@apache.org.
SAMZA-1087: Schedule after debounce time

SAMZA-1087: Allows scheduling an action (a Runnable) after some de-bounce delay.

Author: Boris Shkolnik <bs...@bshkolni-ld1.linkedin.biz>
Author: Boris Shkolnik <bo...@apache.org>

Reviewers: Navina Ramesh <na...@apache.org>, Fred Ji <fj...@linkedin.com>

Closes #49 from sborya/ScheduleAfterDebounceTime1


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

Branch: refs/heads/samza-fluent-api-v1
Commit: 09bf8339ce7ed893c3a2971d49c91fda845f1675
Parents: 2226e3e
Author: Boris Shkolnik <bo...@apache.org>
Authored: Wed Feb 15 17:17:01 2017 -0800
Committer: navina <na...@apache.org>
Committed: Wed Feb 15 17:17:01 2017 -0800

----------------------------------------------------------------------
 .../samza/zk/ScheduleAfterDebounceTime.java     |  86 +++++++++++++++
 .../samza/zk/TestScheduleAfterDebounceTime.java | 110 +++++++++++++++++++
 2 files changed, 196 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/09bf8339/samza-core/src/main/java/org/apache/samza/zk/ScheduleAfterDebounceTime.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ScheduleAfterDebounceTime.java b/samza-core/src/main/java/org/apache/samza/zk/ScheduleAfterDebounceTime.java
new file mode 100644
index 0000000..0a4db6d
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/zk/ScheduleAfterDebounceTime.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.zk;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This class allows scheduling a Runnable actions after some debounce time.
+ * When the same action is scheduled it needs to cancel the previous one. To accomplish that we keep the previous
+ * future in a map, keyed by the action name. Here we predefine some actions, which are used in the
+ * ZK based standalone app.
+ */
+public class ScheduleAfterDebounceTime {
+  public static final Logger LOG = LoggerFactory.getLogger(ScheduleAfterDebounceTime.class);
+  public static final long TIMEOUT_MS = 1000 * 10; // timeout to wait for a task to complete
+
+  // Action name when the JobModel version changes
+  public static final String JOB_MODEL_VERSION_CHANGE = "JobModelVersionChange";
+
+  // Action name when the Processor membership changes
+  public static final String ON_PROCESSOR_CHANGE = "OnProcessorChange";
+
+  // Action name when the Processor Data changes
+  public static final String ON_DATA_CHANGE_ON = "OnDataChangeOn";
+
+  public static final int DEBOUNCE_TIME_MS = 2000;
+
+  private final ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(
+      new ThreadFactoryBuilder().setNameFormat("zk-debounce-thread-%d").setDaemon(true).build());
+  private final Map<String, ScheduledFuture> futureHandles = new HashMap<>();
+
+  synchronized public void scheduleAfterDebounceTime(String actionName, long debounceTimeMs, Runnable runnable) {
+    // check if this action has been scheduled already
+    ScheduledFuture sf = futureHandles.get(actionName);
+    if (sf != null && !sf.isDone()) {
+      LOG.info("DEBOUNCE: cancel future for " + actionName);
+      // attempt to cancel
+      if (!sf.cancel(false)) {
+        try {
+          sf.get(TIMEOUT_MS, TimeUnit.MILLISECONDS);
+        } catch (Exception e) {
+          // we ignore the exception
+          LOG.warn("cancel for action " + actionName + " failed with ", e);
+        }
+      }
+      futureHandles.remove(actionName);
+    }
+    // schedule a new task
+    sf = scheduledExecutorService.schedule(runnable, debounceTimeMs, TimeUnit.MILLISECONDS);
+    LOG.info("DEBOUNCE: scheduled " + actionName + " in " + debounceTimeMs);
+    futureHandles.put(actionName, sf);
+  }
+
+  public void stopScheduler() {
+    // shutdown executor service
+    scheduledExecutorService.shutdown();
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/09bf8339/samza-core/src/test/java/org/apache/samza/zk/TestScheduleAfterDebounceTime.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/zk/TestScheduleAfterDebounceTime.java b/samza-core/src/test/java/org/apache/samza/zk/TestScheduleAfterDebounceTime.java
new file mode 100644
index 0000000..e57372f
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/zk/TestScheduleAfterDebounceTime.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.zk;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+
+public class TestScheduleAfterDebounceTime {
+  private static final long DEBOUNCE_TIME = 500;
+  int i = 0;
+  @Before
+  public void setup() {
+
+  }
+
+  class TestObj {
+    public void inc() {
+      i++;
+    }
+    public void setTo(int val) {
+      i = val;
+    }
+    public void doNothing() {
+
+    }
+  }
+  @Test
+  public void testSchedule() {
+    ScheduleAfterDebounceTime debounceTimer = new ScheduleAfterDebounceTime();
+
+    final TestObj testObj = new TestScheduleAfterDebounceTime.TestObj();
+    debounceTimer.scheduleAfterDebounceTime("TEST1", DEBOUNCE_TIME, () ->
+      {
+        testObj.inc();
+      }
+    );
+    // action is delayed
+    Assert.assertEquals(0, i);
+
+    try {
+      Thread.sleep(DEBOUNCE_TIME + 10);
+    } catch (InterruptedException e) {
+      Assert.fail("Sleep was interrupted");
+    }
+
+    // debounce time passed
+    Assert.assertEquals(1, i);
+  }
+
+  @Test
+  public void testCancelAndSchedule() {
+    ScheduleAfterDebounceTime debounceTimer = new ScheduleAfterDebounceTime();
+
+    final TestObj testObj = new TestScheduleAfterDebounceTime.TestObj();
+    debounceTimer.scheduleAfterDebounceTime("TEST1", DEBOUNCE_TIME, () ->
+      {
+        testObj.inc();
+      }
+    );
+    Assert.assertEquals(0, i);
+
+    // next schedule should cancel the previous one with the same name
+    debounceTimer.scheduleAfterDebounceTime("TEST1", 2 * DEBOUNCE_TIME, () ->
+      {
+        testObj.setTo(100);
+      }
+    );
+
+    try {
+      Thread.sleep(DEBOUNCE_TIME + 10);
+    } catch (InterruptedException e) {
+      Assert.fail("Sleep was interrupted");
+    }
+    // still should be the old value
+    Assert.assertEquals(0, i);
+
+    // this schedule should not cancel the previous one, because it has different name
+    debounceTimer.scheduleAfterDebounceTime("TEST2", DEBOUNCE_TIME, () ->
+      {
+        testObj.doNothing();
+      }
+    );
+
+    try {
+      Thread.sleep(3 * DEBOUNCE_TIME + 10);
+    } catch (InterruptedException e) {
+      Assert.fail("Sleep was interrupted");
+    }
+    Assert.assertEquals(100, i);
+  }
+}


[09/14] samza git commit: SAMZA-1073: top-level fluent API

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/test/java/org/apache/samza/example/TestJoinExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/TestJoinExample.java b/samza-operator/src/test/java/org/apache/samza/example/TestJoinExample.java
new file mode 100644
index 0000000..fe6e7e7
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/TestJoinExample.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.example;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamSpec;
+import org.apache.samza.operators.data.InputMessageEnvelope;
+import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.Offset;
+import org.apache.samza.operators.functions.JoinFunction;
+import org.apache.samza.serializers.JsonSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+
+
+/**
+ * Example implementation of unique key-based stream-stream join tasks
+ *
+ */
+public class TestJoinExample  extends TestExampleBase {
+
+  TestJoinExample(Set<SystemStreamPartition> inputs) {
+    super(inputs);
+  }
+
+  class MessageType {
+    String joinKey;
+    List<String> joinFields = new ArrayList<>();
+  }
+
+  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
+    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
+      super(key, data, offset, partition);
+    }
+  }
+
+  MessageStream<JsonMessageEnvelope> joinOutput = null;
+
+  @Override
+  public void init(StreamGraph graph, Config config) {
+
+    for (SystemStream input : inputs.keySet()) {
+      MessageStream<JsonMessageEnvelope> newSource = graph.<Object, Object, InputMessageEnvelope>createInStream(
+          new StreamSpec() {
+            @Override public SystemStream getSystemStream() {
+              return input;
+            }
+
+            @Override public Properties getProperties() {
+              return null;
+            }
+          }, null, null).map(this::getInputMessage);
+      if (joinOutput == null) {
+        joinOutput = newSource;
+      } else {
+        joinOutput = joinOutput.join(newSource, new MyJoinFunction());
+      }
+    }
+
+    joinOutput.sendTo(graph.createOutStream(new StreamSpec() {
+      @Override public SystemStream getSystemStream() {
+        return null;
+      }
+
+      @Override public Properties getProperties() {
+        return null;
+      }
+    }, new StringSerde("UTF-8"), new JsonSerde<>()));
+
+  }
+
+  private JsonMessageEnvelope getInputMessage(InputMessageEnvelope ism) {
+    return new JsonMessageEnvelope(
+        ((MessageType) ism.getMessage()).joinKey,
+        (MessageType) ism.getMessage(),
+        ism.getOffset(),
+        ism.getSystemStreamPartition());
+  }
+
+  class MyJoinFunction implements JoinFunction<String, JsonMessageEnvelope, JsonMessageEnvelope, JsonMessageEnvelope> {
+    JsonMessageEnvelope myJoinResult(JsonMessageEnvelope m1, JsonMessageEnvelope m2) {
+      MessageType newJoinMsg = new MessageType();
+      newJoinMsg.joinKey = m1.getKey();
+      newJoinMsg.joinFields.addAll(m1.getMessage().joinFields);
+      newJoinMsg.joinFields.addAll(m2.getMessage().joinFields);
+      return new JsonMessageEnvelope(m1.getMessage().joinKey, newJoinMsg, null, null);
+    }
+
+    @Override
+    public JsonMessageEnvelope apply(JsonMessageEnvelope message, JsonMessageEnvelope otherMessage) {
+      return this.myJoinResult(message, otherMessage);
+    }
+
+    @Override
+    public String getFirstKey(JsonMessageEnvelope message) {
+      return message.getKey();
+    }
+
+    @Override
+    public String getSecondKey(JsonMessageEnvelope message) {
+      return message.getKey();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/test/java/org/apache/samza/example/TestWindowExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/TestWindowExample.java b/samza-operator/src/test/java/org/apache/samza/example/TestWindowExample.java
new file mode 100644
index 0000000..e08ca20
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/TestWindowExample.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.example;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamSpec;
+import org.apache.samza.operators.data.InputMessageEnvelope;
+import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.data.Offset;
+import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.time.Duration;
+import java.util.function.BiFunction;
+import java.util.Properties;
+import java.util.Set;
+
+
+/**
+ * Example implementation of a simple user-defined tasks w/ window operators
+ *
+ */
+public class TestWindowExample extends TestExampleBase {
+  class MessageType {
+    String field1;
+    String field2;
+  }
+
+  TestWindowExample(Set<SystemStreamPartition> inputs) {
+    super(inputs);
+  }
+
+  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
+
+    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
+      super(key, data, offset, partition);
+    }
+  }
+
+  @Override
+  public void init(StreamGraph graph, Config config) {
+    BiFunction<JsonMessageEnvelope, Integer, Integer> maxAggregator = (m, c) -> c + 1;
+    inputs.keySet().forEach(source -> graph.<Object, Object, InputMessageEnvelope>createInStream(new StreamSpec() {
+      @Override public SystemStream getSystemStream() {
+        return source;
+      }
+
+      @Override public Properties getProperties() {
+        return null;
+      }
+    }, null, null).
+        map(m1 -> new JsonMessageEnvelope(this.myMessageKeyFunction(m1), (MessageType) m1.getMessage(), m1.getOffset(),
+            m1.getSystemStreamPartition())).window(Windows.tumblingWindow(Duration.ofMillis(200), maxAggregator)));
+
+  }
+
+  String myMessageKeyFunction(MessageEnvelope<Object, Object> m) {
+    return m.getKey().toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/test/java/org/apache/samza/operators/BroadcastTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/BroadcastTask.java b/samza-operator/src/test/java/org/apache/samza/operators/BroadcastTask.java
deleted file mode 100644
index 663d98c..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/BroadcastTask.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.operators;
-
-
-import org.apache.samza.operators.data.IncomingSystemMessageEnvelope;
-import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
-import org.apache.samza.operators.data.Offset;
-import org.apache.samza.operators.triggers.Triggers;
-import org.apache.samza.operators.windows.Windows;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.time.Duration;
-import java.util.Map;
-import java.util.function.BiFunction;
-
-
-/**
- * Example implementation of split stream tasks
- *
- */
-public class BroadcastTask implements StreamOperatorTask {
-  class MessageType {
-    String field1;
-    String field2;
-    String field3;
-    String field4;
-    String parKey;
-    private long timestamp;
-
-    public long getTimestamp() {
-      return this.timestamp;
-    }
-  }
-
-  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
-    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
-      super(key, data, offset, partition);
-    }
-  }
-
-  @Override
-  public void transform(Map<SystemStreamPartition, MessageStream<IncomingSystemMessageEnvelope>> messageStreams) {
-    BiFunction<JsonMessageEnvelope, Integer, Integer> sumAggregator = (m, c) -> c + 1;
-    messageStreams.values().forEach(entry -> {
-        MessageStream<JsonMessageEnvelope> inputStream = entry.map(this::getInputMessage);
-
-        inputStream.filter(this::myFilter1)
-          .window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
-            .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
-
-        inputStream.filter(this::myFilter1)
-          .window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
-            .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
-
-        inputStream.filter(this::myFilter1)
-          .window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
-            .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
-      });
-  }
-
-  JsonMessageEnvelope getInputMessage(IncomingSystemMessageEnvelope m1) {
-    return (JsonMessageEnvelope) m1.getMessage();
-  }
-
-  boolean myFilter1(JsonMessageEnvelope m1) {
-    // Do user defined processing here
-    return m1.getMessage().parKey.equals("key1");
-  }
-
-  boolean myFilter2(JsonMessageEnvelope m1) {
-    // Do user defined processing here
-    return m1.getMessage().parKey.equals("key2");
-  }
-
-  boolean myFilter3(JsonMessageEnvelope m1) {
-    return m1.getMessage().parKey.equals("key3");
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/test/java/org/apache/samza/operators/JoinTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/JoinTask.java b/samza-operator/src/test/java/org/apache/samza/operators/JoinTask.java
deleted file mode 100644
index 1b10609..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/JoinTask.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.operators;
-
-import org.apache.samza.operators.data.IncomingSystemMessageEnvelope;
-import org.apache.samza.operators.data.Offset;
-import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-
-/**
- * Example implementation of unique key-based stream-stream join tasks
- *
- */
-public class JoinTask implements StreamOperatorTask {
-  class MessageType {
-    String joinKey;
-    List<String> joinFields = new ArrayList<>();
-  }
-
-  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
-    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
-      super(key, data, offset, partition);
-    }
-  }
-
-  MessageStream<JsonMessageEnvelope> joinOutput = null;
-
-  @Override
-  public void transform(Map<SystemStreamPartition, MessageStream<IncomingSystemMessageEnvelope>> messageStreams) {
-    messageStreams.values().forEach(messageStream -> {
-        MessageStream<JsonMessageEnvelope> newSource = messageStream.map(this::getInputMessage);
-        if (joinOutput == null) {
-          joinOutput = newSource;
-        } else {
-          joinOutput = joinOutput.join(newSource, (m1, m2) -> this.myJoinResult(m1, m2));
-        }
-      });
-  }
-
-  private JsonMessageEnvelope getInputMessage(IncomingSystemMessageEnvelope ism) {
-    return new JsonMessageEnvelope(
-        ((MessageType) ism.getMessage()).joinKey,
-        (MessageType) ism.getMessage(),
-        ism.getOffset(),
-        ism.getSystemStreamPartition());
-  }
-
-  JsonMessageEnvelope myJoinResult(JsonMessageEnvelope m1, JsonMessageEnvelope m2) {
-    MessageType newJoinMsg = new MessageType();
-    newJoinMsg.joinKey = m1.getKey();
-    newJoinMsg.joinFields.addAll(m1.getMessage().joinFields);
-    newJoinMsg.joinFields.addAll(m2.getMessage().joinFields);
-    return new JsonMessageEnvelope(m1.getMessage().joinKey, newJoinMsg, null, null);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamAdaptorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamAdaptorTask.java b/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamAdaptorTask.java
deleted file mode 100644
index 61bb32a..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamAdaptorTask.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators;
-
-import org.apache.samza.Partition;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.impl.OperatorImpl;
-import org.apache.samza.system.SystemStreamPartition;
-import org.apache.samza.task.TaskContext;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-import java.lang.reflect.Field;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-
-public class TestFluentStreamAdaptorTask {
-  Field userTaskField = null;
-  Field operatorChainsField = null;
-
-  @Before
-  public void prep() throws NoSuchFieldException {
-    userTaskField = StreamOperatorAdaptorTask.class.getDeclaredField("userTask");
-    operatorChainsField = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
-    userTaskField.setAccessible(true);
-    operatorChainsField.setAccessible(true);
-  }
-
-  @Test
-  public void testConstructor() throws IllegalAccessException {
-    StreamOperatorTask userTask = mock(StreamOperatorTask.class);
-    StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(userTask);
-    StreamOperatorTask taskMemberVar = (StreamOperatorTask) userTaskField.get(adaptorTask);
-    Map<SystemStreamPartition, OperatorImpl> chainsMap = (Map<SystemStreamPartition, OperatorImpl>) operatorChainsField.get(adaptorTask);
-    assertEquals(taskMemberVar, userTask);
-    assertTrue(chainsMap.isEmpty());
-  }
-
-  @Test
-  public void testInit() throws Exception {
-    StreamOperatorTask userTask = mock(StreamOperatorTask.class);
-    StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(userTask);
-    Config mockConfig = mock(Config.class);
-    TaskContext mockContext = mock(TaskContext.class);
-    Set<SystemStreamPartition> testInputs = new HashSet() { {
-        this.add(new SystemStreamPartition("test-sys", "test-strm", new Partition(0)));
-        this.add(new SystemStreamPartition("test-sys", "test-strm", new Partition(1)));
-      } };
-    when(mockContext.getSystemStreamPartitions()).thenReturn(testInputs);
-    adaptorTask.init(mockConfig, mockContext);
-    verify(userTask, times(1)).transform(Mockito.anyMap());
-    Map<SystemStreamPartition, OperatorImpl> chainsMap = (Map<SystemStreamPartition, OperatorImpl>) operatorChainsField.get(adaptorTask);
-    assertTrue(chainsMap.size() == 2);
-    assertTrue(chainsMap.containsKey(testInputs.toArray()[0]));
-    assertTrue(chainsMap.containsKey(testInputs.toArray()[1]));
-  }
-
-  // TODO: window and process methods to be added after implementation of ChainedOperators.create()
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamTasks.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamTasks.java b/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamTasks.java
deleted file mode 100644
index d804bf8..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamTasks.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators;
-
-import org.apache.samza.Partition;
-import org.apache.samza.config.Config;
-
-import org.apache.samza.operators.impl.OperatorImpl;
-import org.apache.samza.system.SystemStreamPartition;
-import org.apache.samza.task.TaskContext;
-import org.junit.Test;
-
-import java.lang.reflect.Field;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-
-/**
- * Unit test for {@link StreamOperatorTask}
- */
-public class TestFluentStreamTasks {
-
-  private final WindowTask userTask = new WindowTask();
-
-  private final BroadcastTask splitTask = new BroadcastTask();
-
-  private final JoinTask joinTask = new JoinTask();
-
-  private final Set<SystemStreamPartition> inputPartitions = new HashSet<SystemStreamPartition>() { {
-      for (int i = 0; i < 4; i++) {
-        this.add(new SystemStreamPartition("my-system", "my-topic1", new Partition(i)));
-      }
-    } };
-
-  @Test
-  public void testUserTask() throws Exception {
-    Config mockConfig = mock(Config.class);
-    TaskContext mockContext = mock(TaskContext.class);
-    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
-    StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(this.userTask);
-    Field pipelineMapFld = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
-    pipelineMapFld.setAccessible(true);
-    Map<SystemStreamPartition, OperatorImpl> pipelineMap =
-        (Map<SystemStreamPartition, OperatorImpl>) pipelineMapFld.get(adaptorTask);
-
-    adaptorTask.init(mockConfig, mockContext);
-    assertEquals(pipelineMap.size(), 4);
-    this.inputPartitions.forEach(partition -> {
-        assertNotNull(pipelineMap.get(partition));
-      });
-  }
-
-  @Test
-  public void testSplitTask() throws Exception {
-    Config mockConfig = mock(Config.class);
-    TaskContext mockContext = mock(TaskContext.class);
-    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
-    StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(this.splitTask);
-    Field pipelineMapFld = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
-    pipelineMapFld.setAccessible(true);
-    Map<SystemStreamPartition, OperatorImpl> pipelineMap =
-        (Map<SystemStreamPartition, OperatorImpl>) pipelineMapFld.get(adaptorTask);
-
-    adaptorTask.init(mockConfig, mockContext);
-    assertEquals(pipelineMap.size(), 4);
-    this.inputPartitions.forEach(partition -> {
-        assertNotNull(pipelineMap.get(partition));
-      });
-  }
-
-  @Test
-  public void testJoinTask() throws Exception {
-    Config mockConfig = mock(Config.class);
-    TaskContext mockContext = mock(TaskContext.class);
-    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
-    StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(this.joinTask);
-    Field pipelineMapFld = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
-    pipelineMapFld.setAccessible(true);
-    Map<SystemStreamPartition, OperatorImpl> pipelineMap =
-        (Map<SystemStreamPartition, OperatorImpl>) pipelineMapFld.get(adaptorTask);
-
-    adaptorTask.init(mockConfig, mockContext);
-    assertEquals(pipelineMap.size(), 4);
-    this.inputPartitions.forEach(partition -> {
-        assertNotNull(pipelineMap.get(partition));
-      });
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
index 5991e2f..160a47a 100644
--- a/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
+++ b/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
@@ -29,6 +29,7 @@ import org.apache.samza.operators.spec.SinkOperatorSpec;
 import org.apache.samza.operators.spec.StreamOperatorSpec;
 import org.apache.samza.system.OutgoingMessageEnvelope;
 import org.apache.samza.system.SystemStream;
+import org.apache.samza.task.MessageCollector;
 import org.apache.samza.task.TaskCoordinator;
 import org.junit.Test;
 
@@ -46,17 +47,19 @@ import static org.mockito.Mockito.when;
 
 public class TestMessageStreamImpl {
 
+  private StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+
   @Test
   public void testMap() {
-    MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>();
-    MapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> xMap =
-        m -> new TestOutputMessageEnvelope(m.getKey(), m.getMessage().getValue().length() + 1);
+    MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>(mockGraph);
+    MapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> xMap = (TestMessageEnvelope m)  ->
+        new TestOutputMessageEnvelope(m.getKey(), m.getMessage().getValue().length() + 1);
     MessageStream<TestOutputMessageEnvelope> outputStream = inputStream.map(xMap);
     Collection<OperatorSpec> subs = inputStream.getRegisteredOperatorSpecs();
     assertEquals(subs.size(), 1);
     OperatorSpec<TestOutputMessageEnvelope> mapOp = subs.iterator().next();
     assertTrue(mapOp instanceof StreamOperatorSpec);
-    assertEquals(mapOp.getOutputStream(), outputStream);
+    assertEquals(mapOp.getNextStream(), outputStream);
     // assert that the transformation function is what we defined above
     TestMessageEnvelope xTestMsg = mock(TestMessageEnvelope.class);
     TestMessageEnvelope.MessageType mockInnerTestMessage = mock(TestMessageEnvelope.MessageType.class);
@@ -73,33 +76,33 @@ public class TestMessageStreamImpl {
 
   @Test
   public void testFlatMap() {
-    MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>();
+    MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>(mockGraph);
     Set<TestOutputMessageEnvelope> flatOuts = new HashSet<TestOutputMessageEnvelope>() { {
         this.add(mock(TestOutputMessageEnvelope.class));
         this.add(mock(TestOutputMessageEnvelope.class));
         this.add(mock(TestOutputMessageEnvelope.class));
       } };
-    FlatMapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> xFlatMap = m -> flatOuts;
+    FlatMapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> xFlatMap = (TestMessageEnvelope message) -> flatOuts;
     MessageStream<TestOutputMessageEnvelope> outputStream = inputStream.flatMap(xFlatMap);
     Collection<OperatorSpec> subs = inputStream.getRegisteredOperatorSpecs();
     assertEquals(subs.size(), 1);
     OperatorSpec<TestOutputMessageEnvelope> flatMapOp = subs.iterator().next();
     assertTrue(flatMapOp instanceof StreamOperatorSpec);
-    assertEquals(flatMapOp.getOutputStream(), outputStream);
+    assertEquals(flatMapOp.getNextStream(), outputStream);
     // assert that the transformation function is what we defined above
     assertEquals(((StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope>) flatMapOp).getTransformFn(), xFlatMap);
   }
 
   @Test
   public void testFilter() {
-    MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>();
-    FilterFunction<TestMessageEnvelope> xFilter = m -> m.getMessage().getEventTime() > 123456L;
+    MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>(mockGraph);
+    FilterFunction<TestMessageEnvelope> xFilter = (TestMessageEnvelope m) -> m.getMessage().getEventTime() > 123456L;
     MessageStream<TestMessageEnvelope> outputStream = inputStream.filter(xFilter);
     Collection<OperatorSpec> subs = inputStream.getRegisteredOperatorSpecs();
     assertEquals(subs.size(), 1);
     OperatorSpec<TestMessageEnvelope> filterOp = subs.iterator().next();
     assertTrue(filterOp instanceof StreamOperatorSpec);
-    assertEquals(filterOp.getOutputStream(), outputStream);
+    assertEquals(filterOp.getNextStream(), outputStream);
     // assert that the transformation function is what we defined above
     FlatMapFunction<TestMessageEnvelope, TestMessageEnvelope> txfmFn = ((StreamOperatorSpec<TestMessageEnvelope, TestMessageEnvelope>) filterOp).getTransformFn();
     TestMessageEnvelope mockMsg = mock(TestMessageEnvelope.class);
@@ -117,8 +120,8 @@ public class TestMessageStreamImpl {
 
   @Test
   public void testSink() {
-    MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>();
-    SinkFunction<TestMessageEnvelope> xSink = (m, mc, tc) -> {
+    MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>(mockGraph);
+    SinkFunction<TestMessageEnvelope> xSink = (TestMessageEnvelope m, MessageCollector mc, TaskCoordinator tc) -> {
       mc.send(new OutgoingMessageEnvelope(new SystemStream("test-sys", "test-stream"), m.getMessage()));
       tc.commit(TaskCoordinator.RequestScope.CURRENT_TASK);
     };
@@ -128,26 +131,42 @@ public class TestMessageStreamImpl {
     OperatorSpec<TestMessageEnvelope> sinkOp = subs.iterator().next();
     assertTrue(sinkOp instanceof SinkOperatorSpec);
     assertEquals(((SinkOperatorSpec) sinkOp).getSinkFn(), xSink);
-    assertNull(((SinkOperatorSpec) sinkOp).getOutputStream());
+    assertNull(((SinkOperatorSpec) sinkOp).getNextStream());
   }
 
   @Test
   public void testJoin() {
-    MessageStreamImpl<TestMessageEnvelope> source1 = new MessageStreamImpl<>();
-    MessageStreamImpl<TestMessageEnvelope> source2 = new MessageStreamImpl<>();
-    JoinFunction<TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope> joiner =
-        (m1, m2) -> new TestOutputMessageEnvelope(m1.getKey(), m1.getMessage().getValue().length() + m2.getMessage().getValue().length());
+    MessageStreamImpl<TestMessageEnvelope> source1 = new MessageStreamImpl<>(mockGraph);
+    MessageStreamImpl<TestMessageEnvelope> source2 = new MessageStreamImpl<>(mockGraph);
+    JoinFunction<String, TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope> joiner =
+      new JoinFunction<String, TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope>() {
+        @Override
+        public TestOutputMessageEnvelope apply(TestMessageEnvelope m1, TestMessageEnvelope m2) {
+          return new TestOutputMessageEnvelope(m1.getKey(), m1.getMessage().getValue().length() + m2.getMessage().getValue().length());
+        }
+
+        @Override
+        public String getFirstKey(TestMessageEnvelope message) {
+          return message.getKey();
+        }
+
+        @Override
+        public String getSecondKey(TestMessageEnvelope message) {
+          return message.getKey();
+        }
+      };
+
     MessageStream<TestOutputMessageEnvelope> joinOutput = source1.join(source2, joiner);
     Collection<OperatorSpec> subs = source1.getRegisteredOperatorSpecs();
     assertEquals(subs.size(), 1);
     OperatorSpec<TestMessageEnvelope> joinOp1 = subs.iterator().next();
     assertTrue(joinOp1 instanceof PartialJoinOperatorSpec);
-    assertEquals(((PartialJoinOperatorSpec) joinOp1).getOutputStream(), joinOutput);
+    assertEquals(((PartialJoinOperatorSpec) joinOp1).getNextStream(), joinOutput);
     subs = source2.getRegisteredOperatorSpecs();
     assertEquals(subs.size(), 1);
     OperatorSpec<TestMessageEnvelope> joinOp2 = subs.iterator().next();
     assertTrue(joinOp2 instanceof PartialJoinOperatorSpec);
-    assertEquals(((PartialJoinOperatorSpec) joinOp2).getOutputStream(), joinOutput);
+    assertEquals(((PartialJoinOperatorSpec) joinOp2).getNextStream(), joinOutput);
     TestMessageEnvelope joinMsg1 = new TestMessageEnvelope("test-join-1", "join-msg-001", 11111L);
     TestMessageEnvelope joinMsg2 = new TestMessageEnvelope("test-join-2", "join-msg-002", 22222L);
     TestOutputMessageEnvelope xOut = (TestOutputMessageEnvelope) ((PartialJoinOperatorSpec) joinOp1).getTransformFn().apply(joinMsg1, joinMsg2);
@@ -160,10 +179,10 @@ public class TestMessageStreamImpl {
 
   @Test
   public void testMerge() {
-    MessageStream<TestMessageEnvelope> merge1 = new MessageStreamImpl<>();
+    MessageStream<TestMessageEnvelope> merge1 = new MessageStreamImpl<>(mockGraph);
     Collection<MessageStream<TestMessageEnvelope>> others = new ArrayList<MessageStream<TestMessageEnvelope>>() { {
-        this.add(new MessageStreamImpl<>());
-        this.add(new MessageStreamImpl<>());
+        this.add(new MessageStreamImpl<>(mockGraph));
+        this.add(new MessageStreamImpl<>(mockGraph));
       } };
     MessageStream<TestMessageEnvelope> mergeOutput = merge1.merge(others);
     validateMergeOperator(merge1, mergeOutput);
@@ -176,7 +195,7 @@ public class TestMessageStreamImpl {
     assertEquals(subs.size(), 1);
     OperatorSpec<TestMessageEnvelope> mergeOp = subs.iterator().next();
     assertTrue(mergeOp instanceof StreamOperatorSpec);
-    assertEquals(((StreamOperatorSpec) mergeOp).getOutputStream(), mergeOutput);
+    assertEquals(((StreamOperatorSpec) mergeOp).getNextStream(), mergeOutput);
     TestMessageEnvelope mockMsg = mock(TestMessageEnvelope.class);
     Collection<TestMessageEnvelope> outputs = ((StreamOperatorSpec<TestMessageEnvelope, TestMessageEnvelope>) mergeOp).getTransformFn().apply(mockMsg);
     assertEquals(outputs.size(), 1);

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

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/test/java/org/apache/samza/operators/WindowTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/WindowTask.java b/samza-operator/src/test/java/org/apache/samza/operators/WindowTask.java
deleted file mode 100644
index e176063..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/WindowTask.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.operators;
-
-import org.apache.samza.operators.data.IncomingSystemMessageEnvelope;
-import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
-import org.apache.samza.operators.data.MessageEnvelope;
-import org.apache.samza.operators.data.Offset;
-import org.apache.samza.operators.windows.Windows;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.time.Duration;
-import java.util.Map;
-import java.util.function.BiFunction;
-
-
-/**
- * Example implementation of a simple user-defined tasks w/ window operators
- *
- */
-public class WindowTask implements StreamOperatorTask {
-  class MessageType {
-    String field1;
-    String field2;
-  }
-
-  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
-
-    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
-      super(key, data, offset, partition);
-    }
-  }
-
-  @Override public void transform(Map<SystemStreamPartition, MessageStream<IncomingSystemMessageEnvelope>> messageStreams) {
-    BiFunction<JsonMessageEnvelope, Integer, Integer> maxAggregator = (m, c) -> c + 1;
-    messageStreams.values().forEach(source ->
-        source.map(m1 -> new JsonMessageEnvelope(this.myMessageKeyFunction(m1), (MessageType) m1.getMessage(), m1.getOffset(), m1.getSystemStreamPartition()))
-            .window(Windows.tumblingWindow(Duration.ofMillis(200), maxAggregator))
-    );
-  }
-
-  String myMessageKeyFunction(MessageEnvelope<Object, Object> m) {
-    return m.getKey().toString();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java
index bae98e3..02637a3 100644
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java
@@ -18,17 +18,26 @@
  */
 package org.apache.samza.operators.impl;
 
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import org.apache.samza.config.Config;
 import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.StreamGraphImpl;
 import org.apache.samza.operators.TestMessageEnvelope;
+import org.apache.samza.operators.TestMessageStreamImplUtil;
 import org.apache.samza.operators.TestOutputMessageEnvelope;
 import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.functions.JoinFunction;
+import org.apache.samza.operators.functions.PartialJoinFunction;
 import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.operators.spec.OperatorSpec;
 import org.apache.samza.operators.spec.WindowOperatorSpec;
 import org.apache.samza.operators.spec.PartialJoinOperatorSpec;
 import org.apache.samza.operators.spec.SinkOperatorSpec;
 import org.apache.samza.operators.spec.StreamOperatorSpec;
 import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.operators.windows.internal.WindowInternal;
 import org.apache.samza.task.TaskContext;
 import org.junit.Before;
 import org.junit.Test;
@@ -38,7 +47,6 @@ import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.Set;
-import java.util.function.BiFunction;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotSame;
@@ -49,25 +57,46 @@ import static org.mockito.Mockito.when;
 
 public class TestOperatorImpls {
   Field nextOperatorsField = null;
+  Method createOpMethod = null;
+  Method createOpsMethod = null;
 
   @Before
-  public void prep() throws NoSuchFieldException {
+  public void prep() throws NoSuchFieldException, NoSuchMethodException {
     nextOperatorsField = OperatorImpl.class.getDeclaredField("nextOperators");
     nextOperatorsField.setAccessible(true);
+
+    createOpMethod = OperatorGraph.class.getDeclaredMethod("createOperatorImpl", MessageStreamImpl.class,
+        OperatorSpec.class, Config.class, TaskContext.class);
+    createOpMethod.setAccessible(true);
+
+    createOpsMethod = OperatorGraph.class.getDeclaredMethod("createOperatorImpls", MessageStreamImpl.class, Config.class, TaskContext.class);
+    createOpsMethod.setAccessible(true);
   }
-  
+
   @Test
-  public void testCreateOperator() throws NoSuchFieldException, IllegalAccessException {
+  public void testCreateOperator() throws NoSuchFieldException, IllegalAccessException, InvocationTargetException {
     // get window operator
     WindowOperatorSpec mockWnd = mock(WindowOperatorSpec.class);
-    OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope> opImpl = OperatorImpls.createOperatorImpl(mockWnd);
+    WindowInternal<TestMessageEnvelope, String, Integer> windowInternal = new WindowInternal<>(null, null, null, null);
+    when(mockWnd.getWindow()).thenReturn(windowInternal);
+    MessageStreamImpl<TestMessageEnvelope> mockStream = mock(MessageStreamImpl.class);
+    Config mockConfig = mock(Config.class);
+    TaskContext mockContext = mock(TaskContext.class);
+
+    OperatorGraph opGraph = new OperatorGraph();
+    OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope> opImpl = (OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope>)
+        createOpMethod.invoke(opGraph, mockStream, mockWnd, mockConfig, mockContext);
     assertTrue(opImpl instanceof WindowOperatorImpl);
+    Field wndInternalField = WindowOperatorImpl.class.getDeclaredField("window");
+    wndInternalField.setAccessible(true);
+    WindowInternal wndInternal = (WindowInternal) wndInternalField.get(opImpl);
+    assertEquals(wndInternal, windowInternal);
 
     // get simple operator
     StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> mockSimpleOp = mock(StreamOperatorSpec.class);
     FlatMapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> mockTxfmFn = mock(FlatMapFunction.class);
     when(mockSimpleOp.getTransformFn()).thenReturn(mockTxfmFn);
-    opImpl = OperatorImpls.createOperatorImpl(mockSimpleOp);
+    opImpl = (OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope>) createOpMethod.invoke(opGraph, mockStream, mockSimpleOp, mockConfig, mockContext);
     assertTrue(opImpl instanceof StreamOperatorImpl);
     Field txfmFnField = StreamOperatorImpl.class.getDeclaredField("transformFn");
     txfmFnField.setAccessible(true);
@@ -77,7 +106,7 @@ public class TestOperatorImpls {
     SinkFunction<TestMessageEnvelope> sinkFn = (m, mc, tc) -> { };
     SinkOperatorSpec<TestMessageEnvelope> sinkOp = mock(SinkOperatorSpec.class);
     when(sinkOp.getSinkFn()).thenReturn(sinkFn);
-    opImpl = OperatorImpls.createOperatorImpl(sinkOp);
+    opImpl = (OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope>) createOpMethod.invoke(opGraph, mockStream, sinkOp, mockConfig, mockContext);
     assertTrue(opImpl instanceof SinkOperatorImpl);
     Field sinkFnField = SinkOperatorImpl.class.getDeclaredField("sinkFn");
     sinkFnField.setAccessible(true);
@@ -86,28 +115,33 @@ public class TestOperatorImpls {
     // get join operator
     PartialJoinOperatorSpec<TestMessageEnvelope, String, TestMessageEnvelope, TestOutputMessageEnvelope> joinOp = mock(PartialJoinOperatorSpec.class);
     TestOutputMessageEnvelope mockOutput = mock(TestOutputMessageEnvelope.class);
-    BiFunction<TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope> joinFn = (m1, m2) -> mockOutput;
+    PartialJoinFunction<String, TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope> joinFn = mock(PartialJoinFunction.class);
     when(joinOp.getTransformFn()).thenReturn(joinFn);
-    opImpl = OperatorImpls.createOperatorImpl(joinOp);
+    opImpl = (OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope>) createOpMethod.invoke(opGraph, mockStream, joinOp, mockConfig, mockContext);
     assertTrue(opImpl instanceof PartialJoinOperatorImpl);
   }
 
   @Test
-  public void testEmptyChain() {
+  public void testEmptyChain() throws InvocationTargetException, IllegalAccessException {
     // test creation of empty chain
-    MessageStreamImpl<TestMessageEnvelope> testStream = new MessageStreamImpl<>();
+    MessageStreamImpl<TestMessageEnvelope> testStream = mock(MessageStreamImpl.class);
     TaskContext mockContext = mock(TaskContext.class);
-    RootOperatorImpl operatorChain = OperatorImpls.createOperatorImpls(testStream, mockContext);
+    Config mockConfig = mock(Config.class);
+    OperatorGraph opGraph = new OperatorGraph();
+    RootOperatorImpl operatorChain = (RootOperatorImpl) createOpsMethod.invoke(opGraph, testStream, mockConfig, mockContext);
     assertTrue(operatorChain != null);
   }
 
   @Test
-  public void testLinearChain() throws IllegalAccessException {
+  public void testLinearChain() throws IllegalAccessException, InvocationTargetException {
     // test creation of linear chain
-    MessageStreamImpl<TestMessageEnvelope> testInput = new MessageStreamImpl<>();
+    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    MessageStreamImpl<TestMessageEnvelope> testInput = TestMessageStreamImplUtil.<TestMessageEnvelope>getMessageStreamImpl(mockGraph);
     TaskContext mockContext = mock(TaskContext.class);
-    testInput.map(m -> m).window(Windows.tumblingWindow(Duration.ofMillis(1000)));
-    RootOperatorImpl operatorChain = OperatorImpls.createOperatorImpls(testInput, mockContext);
+    Config mockConfig = mock(Config.class);
+    testInput.map(m -> m).window(Windows.keyedSessionWindow(TestMessageEnvelope::getKey, Duration.ofMinutes(10)));
+    OperatorGraph opGraph = new OperatorGraph();
+    RootOperatorImpl operatorChain = (RootOperatorImpl) createOpsMethod.invoke(opGraph, testInput, mockConfig, mockContext);
     Set<OperatorImpl> subsSet = (Set<OperatorImpl>) nextOperatorsField.get(operatorChain);
     assertEquals(subsSet.size(), 1);
     OperatorImpl<TestMessageEnvelope, TestMessageEnvelope> firstOpImpl = subsSet.iterator().next();
@@ -119,13 +153,16 @@ public class TestOperatorImpls {
   }
 
   @Test
-  public void testBroadcastChain() throws IllegalAccessException {
+  public void testBroadcastChain() throws IllegalAccessException, InvocationTargetException {
     // test creation of broadcast chain
-    MessageStreamImpl<TestMessageEnvelope> testInput = new MessageStreamImpl<>();
+    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    MessageStreamImpl<TestMessageEnvelope> testInput = TestMessageStreamImplUtil.<TestMessageEnvelope>getMessageStreamImpl(mockGraph);
     TaskContext mockContext = mock(TaskContext.class);
+    Config mockConfig = mock(Config.class);
     testInput.filter(m -> m.getMessage().getEventTime() > 123456L).flatMap(m -> new ArrayList() { { this.add(m); this.add(m); } });
     testInput.filter(m -> m.getMessage().getEventTime() < 123456L).map(m -> m);
-    RootOperatorImpl operatorChain = OperatorImpls.createOperatorImpls(testInput, mockContext);
+    OperatorGraph opGraph = new OperatorGraph();
+    RootOperatorImpl operatorChain = (RootOperatorImpl) createOpsMethod.invoke(opGraph, testInput, mockConfig, mockContext);
     Set<OperatorImpl> subsSet = (Set<OperatorImpl>) nextOperatorsField.get(operatorChain);
     assertEquals(subsSet.size(), 2);
     Iterator<OperatorImpl> iter = subsSet.iterator();
@@ -146,18 +183,36 @@ public class TestOperatorImpls {
   }
 
   @Test
-  public void testJoinChain() throws IllegalAccessException {
+  public void testJoinChain() throws IllegalAccessException, InvocationTargetException {
     // test creation of join chain
-    MessageStreamImpl<TestMessageEnvelope> input1 = new MessageStreamImpl<>();
-    MessageStreamImpl<TestMessageEnvelope> input2 = new MessageStreamImpl<>();
+    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    MessageStreamImpl<TestMessageEnvelope> input1 = TestMessageStreamImplUtil.getMessageStreamImpl(mockGraph);
+    MessageStreamImpl<TestMessageEnvelope> input2 = TestMessageStreamImplUtil.getMessageStreamImpl(mockGraph);
     TaskContext mockContext = mock(TaskContext.class);
+    Config mockConfig = mock(Config.class);
     input1
-        .join(input2, (m1, m2) ->
-            new TestOutputMessageEnvelope(m1.getKey(), m1.getMessage().getValue().length() + m2.getMessage().getValue().length()))
+        .join(input2,
+            new JoinFunction<String, TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope>() {
+              @Override
+              public TestOutputMessageEnvelope apply(TestMessageEnvelope m1, TestMessageEnvelope m2) {
+                return new TestOutputMessageEnvelope(m1.getKey(), m1.getMessage().getValue().length() + m2.getMessage().getValue().length());
+              }
+
+              @Override
+              public String getFirstKey(TestMessageEnvelope message) {
+                return message.getKey();
+              }
+
+              @Override
+              public String getSecondKey(TestMessageEnvelope message) {
+                return message.getKey();
+              }
+            })
         .map(m -> m);
+    OperatorGraph opGraph = new OperatorGraph();
     // now, we create chained operators from each input sources
-    RootOperatorImpl chain1 = OperatorImpls.createOperatorImpls(input1, mockContext);
-    RootOperatorImpl chain2 = OperatorImpls.createOperatorImpls(input2, mockContext);
+    RootOperatorImpl chain1 = (RootOperatorImpl) createOpsMethod.invoke(opGraph, input1, mockConfig, mockContext);
+    RootOperatorImpl chain2 = (RootOperatorImpl) createOpsMethod.invoke(opGraph, input2, mockConfig, mockContext);
     // check that those two chains will merge at map operator
     // first branch of the join
     Set<OperatorImpl> subsSet = (Set<OperatorImpl>) nextOperatorsField.get(chain1);

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
index ba5b6f8..ce9fdd2 100644
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
@@ -18,17 +18,16 @@
  */
 package org.apache.samza.operators.impl;
 
+import org.apache.samza.config.Config;
 import org.apache.samza.operators.TestOutputMessageEnvelope;
 import org.apache.samza.operators.functions.SinkFunction;
 import org.apache.samza.operators.spec.SinkOperatorSpec;
 import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
 import org.junit.Test;
 
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.*;
 
 
 public class TestSinkOperatorImpl {
@@ -38,7 +37,9 @@ public class TestSinkOperatorImpl {
     SinkOperatorSpec<TestOutputMessageEnvelope> sinkOp = mock(SinkOperatorSpec.class);
     SinkFunction<TestOutputMessageEnvelope> sinkFn = mock(SinkFunction.class);
     when(sinkOp.getSinkFn()).thenReturn(sinkFn);
-    SinkOperatorImpl<TestOutputMessageEnvelope> sinkImpl = new SinkOperatorImpl<>(sinkOp);
+    Config mockConfig = mock(Config.class);
+    TaskContext mockContext = mock(TaskContext.class);
+    SinkOperatorImpl<TestOutputMessageEnvelope> sinkImpl = new SinkOperatorImpl<>(sinkOp, mockConfig, mockContext);
     TestOutputMessageEnvelope mockMsg = mock(TestOutputMessageEnvelope.class);
     MessageCollector mockCollector = mock(MessageCollector.class);
     TaskCoordinator mockCoordinator = mock(TaskCoordinator.class);

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java
index 5a3840c..010a210 100644
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java
@@ -18,22 +18,20 @@
  */
 package org.apache.samza.operators.impl;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
 import org.apache.samza.operators.TestMessageEnvelope;
 import org.apache.samza.operators.TestOutputMessageEnvelope;
 import org.apache.samza.operators.functions.FlatMapFunction;
 import org.apache.samza.operators.spec.StreamOperatorSpec;
 import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
 import org.junit.Test;
 
-import java.util.ArrayList;
-import java.util.Collection;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.*;
 
 
 public class TestStreamOperatorImpl {
@@ -43,8 +41,10 @@ public class TestStreamOperatorImpl {
     StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> mockOp = mock(StreamOperatorSpec.class);
     FlatMapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> txfmFn = mock(FlatMapFunction.class);
     when(mockOp.getTransformFn()).thenReturn(txfmFn);
-
-    StreamOperatorImpl<TestMessageEnvelope, TestOutputMessageEnvelope> opImpl = spy(new StreamOperatorImpl<>(mockOp));
+    MessageStreamImpl<TestMessageEnvelope> mockInput = mock(MessageStreamImpl.class);
+    Config mockConfig = mock(Config.class);
+    TaskContext mockContext = mock(TaskContext.class);
+    StreamOperatorImpl<TestMessageEnvelope, TestOutputMessageEnvelope> opImpl = spy(new StreamOperatorImpl<>(mockOp, mockInput, mockConfig, mockContext));
     TestMessageEnvelope inMsg = mock(TestMessageEnvelope.class);
     TestOutputMessageEnvelope outMsg = mock(TestOutputMessageEnvelope.class);
     Collection<TestOutputMessageEnvelope> mockOutputs = new ArrayList() { {

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java b/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java
index ffe9df1..31257a4 100644
--- a/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java
+++ b/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java
@@ -18,14 +18,18 @@
  */
 package org.apache.samza.operators.spec;
 
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.StreamGraphImpl;
 import org.apache.samza.operators.TestMessageEnvelope;
+import org.apache.samza.operators.TestMessageStreamImplUtil;
 import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.functions.PartialJoinFunction;
 import org.apache.samza.operators.functions.SinkFunction;
-import org.apache.samza.operators.MessageStreamImpl;
 import org.apache.samza.operators.windows.internal.WindowInternal;
-import org.apache.samza.operators.windows.WindowKey;
 import org.apache.samza.operators.windows.WindowPane;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
 import org.junit.Test;
 
 import java.util.ArrayList;
@@ -42,19 +46,23 @@ public class TestOperatorSpecs {
   @Test
   public void testGetStreamOperator() {
     FlatMapFunction<MessageEnvelope, TestMessageEnvelope> transformFn = m -> new ArrayList<TestMessageEnvelope>() { {
-        this.add(new TestMessageEnvelope(m.getKey().toString(), m.getMessage().toString(), 12345L));
-      } };
-    StreamOperatorSpec<MessageEnvelope, TestMessageEnvelope> strmOp = OperatorSpecs.createStreamOperatorSpec(transformFn);
+          this.add(new TestMessageEnvelope(m.getKey().toString(), m.getMessage().toString(), 12345L));
+        } };
+    MessageStreamImpl<TestMessageEnvelope> mockOutput = mock(MessageStreamImpl.class);
+    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    StreamOperatorSpec<MessageEnvelope, TestMessageEnvelope> strmOp = OperatorSpecs.createStreamOperatorSpec(transformFn, mockGraph, mockOutput);
     assertEquals(strmOp.getTransformFn(), transformFn);
-    assertTrue(strmOp.getOutputStream() instanceof MessageStreamImpl);
+    assertEquals(strmOp.getNextStream(), mockOutput);
   }
 
   @Test
   public void testGetSinkOperator() {
-    SinkFunction<TestMessageEnvelope> sinkFn = (m, c, t) -> { };
-    SinkOperatorSpec<TestMessageEnvelope> sinkOp = OperatorSpecs.createSinkOperatorSpec(sinkFn);
+    SinkFunction<TestMessageEnvelope> sinkFn = (TestMessageEnvelope message, MessageCollector messageCollector,
+          TaskCoordinator taskCoordinator) -> { };
+    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    SinkOperatorSpec<TestMessageEnvelope> sinkOp = OperatorSpecs.createSinkOperatorSpec(sinkFn, mockGraph);
     assertEquals(sinkOp.getSinkFn(), sinkFn);
-    assertTrue(sinkOp.getOutputStream() == null);
+    assertTrue(sinkOp.getNextStream() == null);
   }
 
   @Test
@@ -65,8 +73,9 @@ public class TestOperatorSpecs {
     //instantiate a window using reflection
     WindowInternal window = new WindowInternal(null, aggregator, keyExtractor, null);
 
-    WindowOperatorSpec spec = OperatorSpecs.<TestMessageEnvelope, String, WindowKey<String>, Integer,
-        WindowPane<WindowKey<String>, Integer>>createWindowOperatorSpec(window);
+    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    MessageStreamImpl<WindowPane<String, Integer>> mockWndOut = mock(MessageStreamImpl.class);
+    WindowOperatorSpec spec = OperatorSpecs.<TestMessageEnvelope, String, Integer>createWindowOperatorSpec(window, mockGraph, mockWndOut);
     assertEquals(spec.getWindow(), window);
     assertEquals(spec.getWindow().getKeyExtractor(), keyExtractor);
     assertEquals(spec.getWindow().getFoldFunction(), aggregator);
@@ -74,13 +83,30 @@ public class TestOperatorSpecs {
 
   @Test
   public void testGetPartialJoinOperator() {
-    BiFunction<MessageEnvelope<Object, ?>, MessageEnvelope<Object, ?>, TestMessageEnvelope> merger =
-        (m1, m2) -> new TestMessageEnvelope(m1.getKey().toString(), m2.getMessage().toString(), System.nanoTime());
-    MessageStreamImpl<TestMessageEnvelope> joinOutput = new MessageStreamImpl<>();
+    PartialJoinFunction<Object, MessageEnvelope<Object, ?>, MessageEnvelope<Object, ?>, TestMessageEnvelope> merger =
+      new PartialJoinFunction<Object, MessageEnvelope<Object, ?>, MessageEnvelope<Object, ?>, TestMessageEnvelope>() {
+        @Override
+        public TestMessageEnvelope apply(MessageEnvelope<Object, ?> m1, MessageEnvelope<Object, ?> m2) {
+          return new TestMessageEnvelope(m1.getKey().toString(), m2.getMessage().toString(), System.nanoTime());
+        }
+
+        @Override
+        public Object getKey(MessageEnvelope<Object, ?> message) {
+          return message.getKey();
+        }
+
+        @Override
+        public Object getOtherKey(MessageEnvelope<Object, ?> message) {
+          return message.getKey();
+        }
+      };
+
+    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    MessageStreamImpl<TestMessageEnvelope> joinOutput = TestMessageStreamImplUtil.<TestMessageEnvelope>getMessageStreamImpl(mockGraph);
     PartialJoinOperatorSpec<MessageEnvelope<Object, ?>, Object, MessageEnvelope<Object, ?>, TestMessageEnvelope> partialJoin =
-        OperatorSpecs.createPartialJoinOperatorSpec(merger, joinOutput);
+        OperatorSpecs.createPartialJoinOperatorSpec(merger, mockGraph, joinOutput);
 
-    assertEquals(partialJoin.getOutputStream(), joinOutput);
+    assertEquals(partialJoin.getNextStream(), joinOutput);
     MessageEnvelope<Object, Object> m = mock(MessageEnvelope.class);
     MessageEnvelope<Object, Object> s = mock(MessageEnvelope.class);
     assertEquals(partialJoin.getTransformFn(), merger);
@@ -88,13 +114,14 @@ public class TestOperatorSpecs {
 
   @Test
   public void testGetMergeOperator() {
-    MessageStreamImpl<TestMessageEnvelope> output = new MessageStreamImpl<>();
-    StreamOperatorSpec<TestMessageEnvelope, TestMessageEnvelope> mergeOp = OperatorSpecs.createMergeOperatorSpec(output);
+    StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+    MessageStreamImpl<TestMessageEnvelope> output = TestMessageStreamImplUtil.<TestMessageEnvelope>getMessageStreamImpl(mockGraph);
+    StreamOperatorSpec<TestMessageEnvelope, TestMessageEnvelope> mergeOp = OperatorSpecs.createMergeOperatorSpec(mockGraph, output);
     Function<TestMessageEnvelope, Collection<TestMessageEnvelope>> mergeFn = t -> new ArrayList<TestMessageEnvelope>() { {
         this.add(t);
       } };
     TestMessageEnvelope t = mock(TestMessageEnvelope.class);
     assertEquals(mergeOp.getTransformFn().apply(t), mergeFn.apply(t));
-    assertEquals(mergeOp.getOutputStream(), output);
+    assertEquals(mergeOp.getNextStream(), output);
   }
 }


[03/14] samza git commit: SAMZA-1073: Addressing review feedbacks. Change StreamGraphFactory to StreamGraphBuilder.

Posted by ni...@apache.org.
SAMZA-1073: Addressing review feedbacks. Change StreamGraphFactory to StreamGraphBuilder.


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

Branch: refs/heads/samza-fluent-api-v1
Commit: 4bde68b4a542ac465e4f536bc548c2d6c4366fae
Parents: b3dd886
Author: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Authored: Tue Feb 14 01:33:23 2017 -0800
Committer: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Committed: Wed Feb 15 15:13:09 2017 -0800

----------------------------------------------------------------------
 .../apache/samza/operators/MessageStream.java   |  16 +--
 .../org/apache/samza/operators/StreamGraph.java |  24 ----
 .../samza/operators/StreamGraphBuilder.java     |  38 ++++++
 .../samza/operators/StreamGraphFactory.java     |  38 ------
 .../operators/functions/FilterFunction.java     |  12 +-
 .../operators/functions/FlatMapFunction.java    |  12 +-
 .../samza/operators/functions/InitFunction.java |  38 ------
 .../operators/functions/InitableFunction.java   |  40 ++++++
 .../samza/operators/functions/JoinFunction.java |  11 +-
 .../functions/KeyValueJoinFunction.java         |  44 -------
 .../samza/operators/functions/MapFunction.java  |  12 +-
 .../samza/operators/functions/SinkFunction.java |  14 +-
 .../apache/samza/operators/windows/Window.java  |   2 +-
 .../samza/system/ExecutionEnvironment.java      |   6 +-
 .../java/org/apache/samza/task/TaskContext.java |   2 +-
 .../samza/operators/windows/TestWindowPane.java |   2 +-
 .../samza/operators/MessageStreamImpl.java      |   5 -
 .../functions/PartialJoinFunction.java          |  11 +-
 .../system/RemoteExecutionEnvironment.java      |  37 ++++++
 .../system/SingleJobExecutionEnvironment.java   |  37 ------
 .../system/StandaloneExecutionEnvironment.java  |  19 ++-
 .../apache/samza/task/StreamOperatorTask.java   |  25 ++--
 .../apache/samza/example/BroadcastGraph.java    | 121 -----------------
 .../org/apache/samza/example/JoinGraph.java     | 118 -----------------
 .../samza/example/KeyValueStoreExample.java     |  14 +-
 .../samza/example/NoContextStreamExample.java   |  15 +--
 .../samza/example/OrderShipmentJoinExample.java |   8 +-
 .../samza/example/PageViewCounterExample.java   |  14 +-
 .../samza/example/RepartitionExample.java       |  15 +--
 .../samza/example/TestBasicStreamGraphs.java    |  99 ++++++++++++++
 .../samza/example/TestBroadcastExample.java     | 113 ++++++++++++++++
 .../apache/samza/example/TestExampleBase.java   |  46 +++++++
 .../samza/example/TestFluentStreamTasks.java    |  99 --------------
 .../apache/samza/example/TestJoinExample.java   | 129 +++++++++++++++++++
 .../apache/samza/example/TestWindowExample.java |  81 ++++++++++++
 .../org/apache/samza/example/WindowGraph.java   |  87 -------------
 .../samza/operators/TestMessageStreamImpl.java  |  19 ++-
 .../samza/operators/impl/TestOperatorImpls.java |  21 ++-
 38 files changed, 687 insertions(+), 757 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/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 87a9fd3..adeb4c8 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
@@ -42,7 +42,7 @@ import java.util.function.Function;
 public interface MessageStream<M> {
 
   /**
-   * Applies the provided 1:1 {@link Function} to messages in this {@link MessageStream} and returns the
+   * Applies the provided 1:1 function to messages in this {@link MessageStream} and returns the
    * transformed {@link MessageStream}.
    *
    * @param mapFn the function to transform a message to another message
@@ -52,7 +52,7 @@ public interface MessageStream<M> {
   <TM> MessageStream<TM> map(MapFunction<M, TM> mapFn);
 
   /**
-   * Applies the provided 1:n {@link Function} to transform a message in this {@link MessageStream}
+   * Applies the provided 1:n function to transform a message in this {@link MessageStream}
    * to n messages in the transformed {@link MessageStream}
    *
    * @param flatMapFn the function to transform a message to zero or more messages
@@ -62,7 +62,7 @@ public interface MessageStream<M> {
   <TM> MessageStream<TM> flatMap(FlatMapFunction<M, TM> flatMapFn);
 
   /**
-   * Applies the provided {@link Function} to messages in this {@link MessageStream} and returns the
+   * Applies the provided function to messages in this {@link MessageStream} and returns the
    * transformed {@link MessageStream}.
    * <p>
    * The {@link Function} is a predicate which determines whether a message in this {@link MessageStream}
@@ -92,16 +92,6 @@ public interface MessageStream<M> {
   void sendTo(OutputStream<M> stream);
 
   /**
-   * Allows sending messages to an intermediate {@link MessageStream}.
-   *
-   * NOTE: the {@code stream} has to be a {@link MessageStream}.
-   *
-   * @param stream  the intermediate {@link MessageStream} to send the message to
-   * @return  the intermediate {@link MessageStream} to consume the messages sent
-   */
-  MessageStream<M> sendThrough(OutputStream<M> stream);
-
-  /**
    * Groups the messages in this {@link MessageStream} according to the provided {@link Window} semantics
    * (e.g. tumbling, sliding or session windows) and returns the transformed {@link MessageStream} of
    * {@link WindowPane}s.

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java b/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java
index 9e6644b..abc9861 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java
@@ -19,8 +19,6 @@
 package org.apache.samza.operators;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.config.Config;
-import org.apache.samza.config.ConfigException;
 import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.serializers.Serde;
 
@@ -93,26 +91,4 @@ public interface StreamGraph {
    */
   StreamGraph withContextManager(ContextManager manager);
 
-  String GRAPH_CONFIG = "job.stream.graph.impl.class";
-  String DEFAULT_GRAPH_IMPL_CLASS = "org.apache.samza.operators.StreamGraphImpl";
-
-  /**
-   * Static method to instantiate the implementation class of {@link StreamGraph}.
-   *
-   * @param config  the {@link Config} object for this job
-   * @return  the {@link StreamGraph} object created
-   */
-  static StreamGraph fromConfig(Config config) {
-
-    try {
-      if (StreamGraph.class.isAssignableFrom(Class.forName(config.get(GRAPH_CONFIG, DEFAULT_GRAPH_IMPL_CLASS)))) {
-        return (StreamGraph) Class.forName(config.get(GRAPH_CONFIG, DEFAULT_GRAPH_IMPL_CLASS)).newInstance();
-      }
-    } catch (Exception e) {
-      throw new ConfigException(String.format("Problem in loading StreamGraphImpl class %s", config.get(GRAPH_CONFIG)), e);
-    }
-    throw new ConfigException(String.format(
-        "Class %s does not implement interface StreamGraphBuilder properly",
-        config.get(GRAPH_CONFIG)));
-  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-api/src/main/java/org/apache/samza/operators/StreamGraphBuilder.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/StreamGraphBuilder.java b/samza-api/src/main/java/org/apache/samza/operators/StreamGraphBuilder.java
new file mode 100644
index 0000000..b415cf8
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/StreamGraphBuilder.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;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.config.Config;
+
+
+/**
+ * This interface defines a factory class that user will implement to create user-defined operator DAG in a {@link StreamGraph} object.
+ */
+@InterfaceStability.Unstable
+public interface StreamGraphBuilder {
+  /**
+   * Users are required to implement this abstract method to initialize the processing logic of the application, in terms
+   * of a DAG of {@link org.apache.samza.operators.MessageStream}s and operators
+   *
+   * @param graph  an empty {@link StreamGraph} object to be initialized
+   * @param config  the {@link Config} of the application
+   */
+  void init(StreamGraph graph, Config config);
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-api/src/main/java/org/apache/samza/operators/StreamGraphFactory.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/StreamGraphFactory.java b/samza-api/src/main/java/org/apache/samza/operators/StreamGraphFactory.java
deleted file mode 100644
index c292363..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/StreamGraphFactory.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators;
-
-import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.config.Config;
-
-
-/**
- * This interface defines a factory class that user will implement to create user-defined operator DAG in a {@link StreamGraph} object.
- */
-@InterfaceStability.Unstable
-public interface StreamGraphFactory {
-  /**
-   * Users are required to implement this abstract method to initialize the processing logic of the application, in terms
-   * of a DAG of {@link org.apache.samza.operators.MessageStream}s and operators
-   *
-   * @param config  the {@link Config} of the application
-   * @return  the {@link StreamGraph} object which contains user-defined processing logic of the application
-   */
-  StreamGraph create(Config config);
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/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 73c5c9d..58479d6 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
@@ -19,8 +19,6 @@
 package org.apache.samza.operators.functions;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.config.Config;
-import org.apache.samza.task.TaskContext;
 
 
 /**
@@ -28,7 +26,8 @@ import org.apache.samza.task.TaskContext;
  * @param <M>  type of the input message
  */
 @InterfaceStability.Unstable
-public interface FilterFunction<M> extends InitFunction {
+@FunctionalInterface
+public interface FilterFunction<M> extends InitableFunction {
 
   /**
    * Returns a boolean indicating whether this message should be retained or filtered out.
@@ -37,11 +36,4 @@ public interface FilterFunction<M> extends InitFunction {
    */
   boolean apply(M message);
 
-  /**
-   * Init method to initialize the context for this {@link FilterFunction}. The default implementation is NO-OP.
-   *
-   * @param config  the {@link Config} object for this task
-   * @param context  the {@link TaskContext} object for this task
-   */
-  default void init(Config config, TaskContext context) { }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/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 f8458f2..bbbddeb 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
@@ -19,8 +19,6 @@
 package org.apache.samza.operators.functions;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.config.Config;
-import org.apache.samza.task.TaskContext;
 
 import java.util.Collection;
 
@@ -32,7 +30,8 @@ import java.util.Collection;
  * @param <OM>  type of the transformed messages
  */
 @InterfaceStability.Unstable
-public interface FlatMapFunction<M, OM>  extends InitFunction {
+@FunctionalInterface
+public interface FlatMapFunction<M, OM>  extends InitableFunction {
 
   /**
    * Transforms the provided message into a collection of 0 or more messages.
@@ -41,11 +40,4 @@ public interface FlatMapFunction<M, OM>  extends InitFunction {
    */
   Collection<OM> apply(M message);
 
-  /**
-   * Init method to initialize the context for this {@link FlatMapFunction}. The default implementation is NO-OP.
-   *
-   * @param config  the {@link Config} object for this task
-   * @param context  the {@link TaskContext} object for this task
-   */
-  default void init(Config config, TaskContext context) { };
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-api/src/main/java/org/apache/samza/operators/functions/InitFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/InitFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/InitFunction.java
deleted file mode 100644
index eec56df..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/functions/InitFunction.java
+++ /dev/null
@@ -1,38 +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.functions;
-
-import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.config.Config;
-import org.apache.samza.task.TaskContext;
-
-
-/**
- * interface defined to initalize the context of message transformation functions
- */
-@InterfaceStability.Unstable
-public interface InitFunction {
-  /**
-   * Interface method to initialize the context for a specific message transformation function.
-   *
-   * @param config  the {@link Config} object for this task
-   * @param context  the {@link TaskContext} object for this task
-   */
-  void init(Config config, TaskContext context);
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java
new file mode 100644
index 0000000..2f738da
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.functions;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.config.Config;
+import org.apache.samza.task.TaskContext;
+
+
+/**
+ * interface defined to initalize the context of message transformation functions
+ */
+@InterfaceStability.Unstable
+public interface InitableFunction {
+
+  /**
+   * Interface method to initialize the context for a specific message transformation function.
+   *
+   * @param config  the {@link Config} object for this task
+   * @param context  the {@link TaskContext} object for this task
+   */
+  default void init(Config config, TaskContext context) { }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/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 afc92ee..fc38177 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
@@ -19,8 +19,6 @@
 package org.apache.samza.operators.functions;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.config.Config;
-import org.apache.samza.task.TaskContext;
 
 
 /**
@@ -32,7 +30,7 @@ import org.apache.samza.task.TaskContext;
  * @param <RM>  type of the joined message
  */
 @InterfaceStability.Unstable
-public interface JoinFunction<K, M, JM, RM>  extends InitFunction {
+public interface JoinFunction<K, M, JM, RM>  extends InitableFunction {
 
   /**
    * Join the provided input messages and produces the joined messages.
@@ -58,11 +56,4 @@ public interface JoinFunction<K, M, JM, RM>  extends InitFunction {
    */
   K getSecondKey(JM message);
 
-  /**
-   * Init method to initialize the context for this {@link JoinFunction}. The default implementation is NO-OP.
-   *
-   * @param config  the {@link Config} object for this task
-   * @param context  the {@link TaskContext} object for this task
-   */
-  default void init(Config config, TaskContext context) { }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-api/src/main/java/org/apache/samza/operators/functions/KeyValueJoinFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/KeyValueJoinFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/KeyValueJoinFunction.java
deleted file mode 100644
index b651b3d..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/functions/KeyValueJoinFunction.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators.functions;
-
-import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.MessageEnvelope;
-
-/**
- * A specific {@link JoinFunction} that joins {@link MessageEnvelope}s from two {@link org.apache.samza.operators.MessageStream}s and produces
- * a joined message.
- *
- * @param <K>  type of the join key
- * @param <M>  type of the input {@link MessageEnvelope}
- * @param <JM>  type of the {@link MessageEnvelope} to join with
- * @param <RM>  type of the joined message
- */
-@InterfaceStability.Unstable
-@FunctionalInterface
-public interface KeyValueJoinFunction<K, M extends MessageEnvelope<K, ?>, JM extends MessageEnvelope<K, ?>, RM> extends JoinFunction<K, M, JM, RM> {
-
-  default K getFirstKey(M message) {
-    return message.getKey();
-  }
-
-  default K getSecondKey(JM message) {
-    return message.getKey();
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/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 a051914..05a554f 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
@@ -19,8 +19,6 @@
 package org.apache.samza.operators.functions;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.config.Config;
-import org.apache.samza.task.TaskContext;
 
 
 /**
@@ -29,7 +27,8 @@ import org.apache.samza.task.TaskContext;
  * @param <OM>  type of the transformed message
  */
 @InterfaceStability.Unstable
-public interface MapFunction<M, OM>  extends InitFunction {
+@FunctionalInterface
+public interface MapFunction<M, OM>  extends InitableFunction {
 
   /**
    * Transforms the provided message into another message
@@ -38,11 +37,4 @@ public interface MapFunction<M, OM>  extends InitFunction {
    */
   OM apply(M message);
 
-  /**
-   * Init method to initialize the context for this {@link MapFunction}. The default implementation is NO-OP.
-   *
-   * @param config  the {@link Config} object for this task
-   * @param context  the {@link TaskContext} object for this task
-   */
-  default void init(Config config, TaskContext context) { }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/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 1050771..08e090a 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
@@ -19,9 +19,7 @@
 package org.apache.samza.operators.functions;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.config.Config;
 import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
 
 
@@ -30,7 +28,8 @@ import org.apache.samza.task.TaskCoordinator;
  * @param <M>  type of the input message
  */
 @InterfaceStability.Unstable
-public interface SinkFunction<M>  extends InitFunction {
+@FunctionalInterface
+public interface SinkFunction<M>  extends InitableFunction {
 
   /**
    * Allows sending the provided message to an output {@link org.apache.samza.system.SystemStream} using
@@ -38,16 +37,9 @@ public interface SinkFunction<M>  extends InitFunction {
    * or shut the container down.
    *
    * @param message  the input message to be sent to an output {@link org.apache.samza.system.SystemStream}
-   * @param messageCollector  the {@link MessageCollector} to use to send the {@link org.apache.samza.operators.data.MessageEnvelope}
+   * @param messageCollector  the {@link MessageCollector} to send the {@link org.apache.samza.operators.data.MessageEnvelope}
    * @param taskCoordinator  the {@link TaskCoordinator} to request commits or shutdown
    */
   void apply(M message, MessageCollector messageCollector, TaskCoordinator taskCoordinator);
 
-  /**
-   * Init method to initialize the context for this {@link MapFunction}. The default implementation is NO-OP.
-   *
-   * @param config  the {@link Config} object for this task
-   * @param context  the {@link TaskContext} object for this task
-   */
-  default void init(Config config, TaskContext context) { }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/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 8aa665a..9609292 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
@@ -29,7 +29,7 @@ import org.apache.samza.operators.triggers.Trigger;
  * that determine when results from the {@link Window} are emitted.
  *
  * <p> Each emitted result contains one or more messages in the window and is called a {@link WindowPane}.
- * A pane can include all messagess collected for the window so far or only the new messages
+ * A pane can include all messages collected for the window so far or only the new messages
  * since the last emitted pane. (as determined by the {@link AccumulationMode})
  *
  * <p> A window can have early triggers that allow emitting {@link WindowPane}s speculatively before all data for the window

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-api/src/main/java/org/apache/samza/system/ExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/ExecutionEnvironment.java b/samza-api/src/main/java/org/apache/samza/system/ExecutionEnvironment.java
index d0c5985..ad37eb3 100644
--- a/samza-api/src/main/java/org/apache/samza/system/ExecutionEnvironment.java
+++ b/samza-api/src/main/java/org/apache/samza/system/ExecutionEnvironment.java
@@ -20,7 +20,7 @@ package org.apache.samza.system;
 
 import org.apache.samza.annotation.InterfaceStability;
 import org.apache.samza.config.ConfigException;
-import org.apache.samza.operators.StreamGraphFactory;
+import org.apache.samza.operators.StreamGraphBuilder;
 import org.apache.samza.config.Config;
 
 
@@ -65,9 +65,9 @@ public interface ExecutionEnvironment {
   /**
    * Method to be invoked to deploy and run the actual Samza jobs to execute {@link org.apache.samza.operators.StreamGraph}
    *
-   * @param graphFactory  the user-defined {@link StreamGraphFactory} object
+   * @param graphBuilder  the user-defined {@link StreamGraphBuilder} object
    * @param config  the {@link Config} object for this job
    */
-  void run(StreamGraphFactory graphFactory, Config config);
+  void run(StreamGraphBuilder graphBuilder, Config config);
 
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-api/src/main/java/org/apache/samza/task/TaskContext.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/task/TaskContext.java b/samza-api/src/main/java/org/apache/samza/task/TaskContext.java
index 5779071..128cff1 100644
--- a/samza-api/src/main/java/org/apache/samza/task/TaskContext.java
+++ b/samza-api/src/main/java/org/apache/samza/task/TaskContext.java
@@ -61,7 +61,7 @@ public interface TaskContext {
    * @param <T>  the type of user-defined task context
    * @return  user-defined task context object
    */
-  default <T extends TaskContext> T getUserDefinedContext() {
+  default <T> T getUserDefinedContext() {
     return null;
   };
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/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 809c5b4..54d0b2f 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
@@ -26,7 +26,7 @@ import static org.junit.Assert.assertEquals;
 public class TestWindowPane {
   @Test
   public void testConstructor() {
-    WindowPane<String, Integer> wndOutput = WindowPane.of(new WindowKey("testMsg", null), 10);
+    WindowPane<String, Integer> wndOutput = WindowPane.of(new WindowKey<>("testMsg", null), 10);
     assertEquals(wndOutput.getKey().getKey(), "testMsg");
     assertEquals(wndOutput.getMessage(), Integer.valueOf(10));
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
index d85d488..830e4a5 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
@@ -93,11 +93,6 @@ public class MessageStreamImpl<M> implements MessageStream<M> {
     this.registeredOperatorSpecs.add(OperatorSpecs.createSendToOperatorSpec(stream.getSinkFunction(), this.graph, stream));
   }
 
-  @Override public MessageStream<M> sendThrough(OutputStream<M> stream) {
-    this.sendTo(stream);
-    return this.graph.getIntStream(stream);
-  }
-
   @Override
   public <K, WV> MessageStream<WindowPane<K, WV>> window(Window<M, K, WV> window) {
     OperatorSpec<WindowPane<K, WV>> wndOp = OperatorSpecs.createWindowOperatorSpec((WindowInternal<M, K, WV>) window,

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java b/samza-operator/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java
index 3583b92..809a70a 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java
@@ -19,8 +19,6 @@
 package org.apache.samza.operators.functions;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.config.Config;
-import org.apache.samza.task.TaskContext;
 
 
 /**
@@ -28,7 +26,7 @@ import org.apache.samza.task.TaskContext;
  * {@link org.apache.samza.operators.MessageStream}s and merge them into a single output joined message in the join output
  */
 @InterfaceStability.Unstable
-public interface PartialJoinFunction<K, M, OM, RM> extends InitFunction {
+public interface PartialJoinFunction<K, M, OM, RM> extends InitableFunction {
 
   /**
    * Method to perform join method on the two input messages
@@ -55,11 +53,4 @@ public interface PartialJoinFunction<K, M, OM, RM> extends InitFunction {
    */
   K getOtherKey(OM message);
 
-  /**
-   * Init method to initialize the context for this {@link PartialJoinFunction}. The default implementation is NO-OP.
-   *
-   * @param config  the {@link Config} object for this task
-   * @param context  the {@link TaskContext} object for this task
-   */
-  default void init(Config config, TaskContext context) { }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java b/samza-operator/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java
new file mode 100644
index 0000000..fafa2cb
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system;
+
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.apache.samza.config.Config;
+
+/**
+ * This class implements the {@link ExecutionEnvironment} that runs the applications in YARN environment
+ */
+public class RemoteExecutionEnvironment implements ExecutionEnvironment {
+
+  @Override public void run(StreamGraphBuilder app, Config config) {
+    // TODO: add description of ProcessContext that is going to create a sub-DAG of the {@code graph}
+    // TODO: actually instantiate the tasks and run the job, i.e.
+    // 1. create all input/output/intermediate topics
+    // 2. create the single job configuration
+    // 3. execute JobRunner to submit the single job for the whole graph
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/main/java/org/apache/samza/system/SingleJobExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/system/SingleJobExecutionEnvironment.java b/samza-operator/src/main/java/org/apache/samza/system/SingleJobExecutionEnvironment.java
deleted file mode 100644
index 60a4c60..0000000
--- a/samza-operator/src/main/java/org/apache/samza/system/SingleJobExecutionEnvironment.java
+++ /dev/null
@@ -1,37 +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.system;
-
-import org.apache.samza.operators.StreamGraphFactory;
-import org.apache.samza.config.Config;
-
-/**
- * This class implements the {@link ExecutionEnvironment} that runs the applications in YARN environment
- */
-public class SingleJobExecutionEnvironment implements ExecutionEnvironment {
-
-  @Override public void run(StreamGraphFactory app, Config config) {
-    // TODO: add description of ProcessContext that is going to create a sub-DAG of the {@code graph}
-    // TODO: actually instantiate the tasks and run the job, i.e.
-    // 1. create all input/output/intermediate topics
-    // 2. create the single job configuration
-    // 3. execute JobRunner to submit the single job for the whole graph
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java b/samza-operator/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java
index f60ff82..f0f6ef2 100644
--- a/samza-operator/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java
+++ b/samza-operator/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java
@@ -19,8 +19,10 @@
 
 package org.apache.samza.system;
 
-import org.apache.samza.operators.StreamGraphFactory;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamGraphBuilder;
 import org.apache.samza.config.Config;
+import org.apache.samza.operators.StreamGraphImpl;
 
 
 /**
@@ -28,14 +30,21 @@ import org.apache.samza.config.Config;
  */
 public class StandaloneExecutionEnvironment implements ExecutionEnvironment {
 
-  @Override public void run(StreamGraphFactory app, Config config) {
+  // TODO: may want to move this to a common base class for all {@link ExecutionEnvironment}
+  StreamGraph createGraph(StreamGraphBuilder app, Config config) {
+    StreamGraphImpl graph = new StreamGraphImpl();
+    app.init(graph, config);
+    return graph;
+  }
+
+  @Override public void run(StreamGraphBuilder app, Config config) {
     // 1. get logic graph for optimization
-    // StreamGraph logicGraph = app.create(config);
+    // StreamGraph logicGraph = this.createGraph(app, config);
     // 2. potential optimization....
-    // 3. create new instance of StreamGraphFactory that would generate the optimized graph
+    // 3. create new instance of StreamGraphBuilder that would generate the optimized graph
     // 4. create all input/output/intermediate topics
     // 5. create the configuration for StreamProcessor
-    // 6. start the StreamProcessor w/ optimized instance of StreamGraphFactory
+    // 6. start the StreamProcessor w/ optimized instance of StreamGraphBuilder
   }
 
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java b/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java
index fa7ec5e..b007e3c 100644
--- a/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java
+++ b/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java
@@ -21,7 +21,7 @@ package org.apache.samza.task;
 import org.apache.samza.config.Config;
 import org.apache.samza.operators.ContextManager;
 import org.apache.samza.operators.MessageStreamImpl;
-import org.apache.samza.operators.StreamGraphFactory;
+import org.apache.samza.operators.StreamGraphBuilder;
 import org.apache.samza.operators.StreamGraphImpl;
 import org.apache.samza.operators.data.InputMessageEnvelope;
 import org.apache.samza.operators.impl.OperatorGraph;
@@ -43,9 +43,9 @@ import java.util.Map;
  * This class brings all the operator API implementation components together and feeds the
  * {@link InputMessageEnvelope}s into the transformation chains.
  * <p>
- * It accepts an instance of the user implemented factory {@link StreamGraphFactory} as input parameter of the constructor.
+ * It accepts an instance of the user implemented factory {@link StreamGraphBuilder} as input parameter of the constructor.
  * When its own {@link #init(Config, TaskContext)} method is called during startup, it instantiate a user-defined {@link StreamGraphImpl}
- * from the {@link StreamGraphFactory}, calls {@link StreamGraphImpl#getContextManager()} to initialize the task-wide context
+ * from the {@link StreamGraphBuilder}, calls {@link StreamGraphImpl#getContextManager()} to initialize the task-wide context
  * for the graph, and creates a {@link MessageStreamImpl} corresponding to each of its input
  * {@link org.apache.samza.system.SystemStreamPartition}s. Each input {@link MessageStreamImpl}
  * will be corresponding to either an input stream or intermediate stream in {@link StreamGraphImpl}.
@@ -67,27 +67,30 @@ public final class StreamOperatorTask implements StreamTask, InitableTask, Windo
    */
   private final OperatorGraph operatorGraph = new OperatorGraph();
 
-  private final StreamGraphFactory graphFactory;
+  private final StreamGraphBuilder graphBuilder;
 
-  private ContextManager taskManager;
+  private ContextManager contextManager;
 
-  public StreamOperatorTask(StreamGraphFactory graphFactory) {
-    this.graphFactory = graphFactory;
+  public StreamOperatorTask(StreamGraphBuilder graphBuilder) {
+    this.graphBuilder = graphBuilder;
   }
 
   @Override
   public final void init(Config config, TaskContext context) throws Exception {
     // create the MessageStreamsImpl object and initialize app-specific logic DAG within the task
-    StreamGraphImpl streams = (StreamGraphImpl) this.graphFactory.create(config);
-    this.taskManager = streams.getContextManager();
+    StreamGraphImpl streams = new StreamGraphImpl();
+    this.graphBuilder.init(streams, config);
+    // get the context manager of the {@link StreamGraph} and initialize the task-specific context
+    this.contextManager = streams.getContextManager();
 
     Map<SystemStream, MessageStreamImpl> inputBySystemStream = new HashMap<>();
     context.getSystemStreamPartitions().forEach(ssp -> {
         if (!inputBySystemStream.containsKey(ssp.getSystemStream())) {
+          // create mapping from the physical input {@link SystemStream} to the logic {@link MessageStream}
           inputBySystemStream.putIfAbsent(ssp.getSystemStream(), streams.getInputStream(ssp.getSystemStream()));
         }
       });
-    operatorGraph.init(inputBySystemStream, config, this.taskManager.initTaskContext(config, context));
+    operatorGraph.init(inputBySystemStream, config, this.contextManager.initTaskContext(config, context));
   }
 
   @Override
@@ -103,6 +106,6 @@ public final class StreamOperatorTask implements StreamTask, InitableTask, Windo
 
   @Override
   public void close() throws Exception {
-    this.taskManager.finalizeTaskContext();
+    this.contextManager.finalizeTaskContext();
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/test/java/org/apache/samza/example/BroadcastGraph.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/BroadcastGraph.java b/samza-operator/src/test/java/org/apache/samza/example/BroadcastGraph.java
deleted file mode 100644
index a91ce09..0000000
--- a/samza-operator/src/test/java/org/apache/samza/example/BroadcastGraph.java
+++ /dev/null
@@ -1,121 +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.config.Config;
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.StreamGraph;
-import org.apache.samza.operators.StreamGraphFactory;
-import org.apache.samza.operators.StreamGraphImpl;
-import org.apache.samza.operators.StreamSpec;
-import org.apache.samza.operators.data.InputMessageEnvelope;
-import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
-import org.apache.samza.operators.data.Offset;
-import org.apache.samza.operators.triggers.Triggers;
-import org.apache.samza.operators.windows.Windows;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.time.Duration;
-import java.util.function.BiFunction;
-import java.util.Properties;
-import java.util.Set;
-
-
-/**
- * Example implementation of split stream tasks
- *
- */
-public class BroadcastGraph implements StreamGraphFactory {
-
-  private final Set<SystemStreamPartition> inputs;
-
-  BroadcastGraph(Set<SystemStreamPartition> inputs) {
-    this.inputs = inputs;
-  }
-
-  class MessageType {
-    String field1;
-    String field2;
-    String field3;
-    String field4;
-    String parKey;
-    private long timestamp;
-
-    public long getTimestamp() {
-      return this.timestamp;
-    }
-  }
-
-  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
-    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
-      super(key, data, offset, partition);
-    }
-  }
-
-  @Override
-  public StreamGraph create(Config config) {
-    StreamGraphImpl graph = new StreamGraphImpl();
-
-    BiFunction<JsonMessageEnvelope, Integer, Integer> sumAggregator = (m, c) -> c + 1;
-    inputs.forEach(entry -> {
-        MessageStream<JsonMessageEnvelope> inputStream = graph.<Object, Object, InputMessageEnvelope>createInStream(new StreamSpec() {
-          @Override public SystemStream getSystemStream() {
-            return entry.getSystemStream();
-          }
-
-          @Override public Properties getProperties() {
-            return null;
-          }
-        }, null, null).
-            map(this::getInputMessage);
-
-        inputStream.filter(this::myFilter1).window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
-            .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
-
-        inputStream.filter(this::myFilter2).window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
-            .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
-
-        inputStream.filter(this::myFilter3).window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
-            .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
-
-      });
-    return graph;
-  }
-
-  JsonMessageEnvelope getInputMessage(InputMessageEnvelope m1) {
-    return (JsonMessageEnvelope) m1.getMessage();
-  }
-
-  boolean myFilter1(JsonMessageEnvelope m1) {
-    // Do user defined processing here
-    return m1.getMessage().parKey.equals("key1");
-  }
-
-  boolean myFilter2(JsonMessageEnvelope m1) {
-    // Do user defined processing here
-    return m1.getMessage().parKey.equals("key2");
-  }
-
-  boolean myFilter3(JsonMessageEnvelope m1) {
-    return m1.getMessage().parKey.equals("key3");
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/test/java/org/apache/samza/example/JoinGraph.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/JoinGraph.java b/samza-operator/src/test/java/org/apache/samza/example/JoinGraph.java
deleted file mode 100644
index 2313f63..0000000
--- a/samza-operator/src/test/java/org/apache/samza/example/JoinGraph.java
+++ /dev/null
@@ -1,118 +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.config.Config;
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.StreamGraph;
-import org.apache.samza.operators.StreamGraphFactory;
-import org.apache.samza.operators.StreamGraphImpl;
-import org.apache.samza.operators.StreamSpec;
-import org.apache.samza.operators.data.InputMessageEnvelope;
-import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
-import org.apache.samza.operators.data.Offset;
-import org.apache.samza.operators.functions.KeyValueJoinFunction;
-import org.apache.samza.serializers.JsonSerde;
-import org.apache.samza.serializers.StringSerde;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Properties;
-import java.util.Set;
-
-
-/**
- * Example implementation of unique key-based stream-stream join tasks
- *
- */
-public class JoinGraph implements StreamGraphFactory {
-  private final Set<SystemStreamPartition> inputs;
-
-  JoinGraph(Set<SystemStreamPartition> inputs) {
-    this.inputs = inputs;
-  }
-
-  class MessageType {
-    String joinKey;
-    List<String> joinFields = new ArrayList<>();
-  }
-
-  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
-    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
-      super(key, data, offset, partition);
-    }
-  }
-
-  MessageStream<JsonMessageEnvelope> joinOutput = null;
-
-  @Override
-  public StreamGraph create(Config config) {
-    StreamGraphImpl graph = new StreamGraphImpl();
-
-    for (SystemStreamPartition input : inputs) {
-      MessageStream<JsonMessageEnvelope> newSource = graph.<Object, Object, InputMessageEnvelope>createInStream(
-          new StreamSpec() {
-            @Override public SystemStream getSystemStream() {
-              return input.getSystemStream();
-            }
-
-            @Override public Properties getProperties() {
-              return null;
-            }
-          }, null, null).map(this::getInputMessage);
-      if (joinOutput == null) {
-        joinOutput = newSource;
-      } else {
-        joinOutput = joinOutput.join(newSource,
-            (KeyValueJoinFunction<String, JsonMessageEnvelope, JsonMessageEnvelope, JsonMessageEnvelope>) this::myJoinResult);
-      }
-    }
-
-    joinOutput.sendTo(graph.createOutStream(new StreamSpec() {
-      @Override public SystemStream getSystemStream() {
-        return null;
-      }
-
-      @Override public Properties getProperties() {
-        return null;
-      }
-    }, new StringSerde("UTF-8"), new JsonSerde<>()));
-
-    return graph;
-  }
-
-  private JsonMessageEnvelope getInputMessage(InputMessageEnvelope ism) {
-    return new JsonMessageEnvelope(
-        ((MessageType) ism.getMessage()).joinKey,
-        (MessageType) ism.getMessage(),
-        ism.getOffset(),
-        ism.getSystemStreamPartition());
-  }
-
-  JsonMessageEnvelope myJoinResult(JsonMessageEnvelope m1, JsonMessageEnvelope m2) {
-    MessageType newJoinMsg = new MessageType();
-    newJoinMsg.joinKey = m1.getKey();
-    newJoinMsg.joinFields.addAll(m1.getMessage().joinFields);
-    newJoinMsg.joinFields.addAll(m2.getMessage().joinFields);
-    return new JsonMessageEnvelope(m1.getMessage().joinKey, newJoinMsg, null, null);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/test/java/org/apache/samza/example/KeyValueStoreExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/KeyValueStoreExample.java b/samza-operator/src/test/java/org/apache/samza/example/KeyValueStoreExample.java
index ad6336a..85ebc6c 100644
--- a/samza-operator/src/test/java/org/apache/samza/example/KeyValueStoreExample.java
+++ b/samza-operator/src/test/java/org/apache/samza/example/KeyValueStoreExample.java
@@ -22,12 +22,10 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.OutputStream;
-import org.apache.samza.operators.StreamGraphFactory;
+
+import org.apache.samza.operators.*;
+import org.apache.samza.operators.StreamGraphBuilder;
 import org.apache.samza.config.Config;
-import org.apache.samza.operators.StreamGraph;
-import org.apache.samza.operators.StreamSpec;
 import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.operators.functions.FlatMapFunction;
 import org.apache.samza.serializers.JsonSerde;
@@ -44,7 +42,7 @@ import java.util.Properties;
 /**
  * Example code using {@link KeyValueStore} to implement event-time window
  */
-public class KeyValueStoreExample implements StreamGraphFactory {
+public class KeyValueStoreExample implements StreamGraphBuilder {
 
   /**
    * used by remote execution environment to launch the job in remote program. The remote program should follow the similar
@@ -59,8 +57,7 @@ public class KeyValueStoreExample implements StreamGraphFactory {
    *   }
    *
    */
-  @Override public StreamGraph create(Config config) {
-    StreamGraph graph = StreamGraph.fromConfig(config);
+  @Override public void init(StreamGraph graph, Config config) {
 
     MessageStream<PageViewEvent> pageViewEvents = graph.createInStream(input1, new StringSerde("UTF-8"), new JsonSerde<>());
     OutputStream<StatsOutput> pageViewPerMemberCounters = graph.createOutStream(output, new StringSerde("UTF-8"), new JsonSerde<StatsOutput>());
@@ -70,7 +67,6 @@ public class KeyValueStoreExample implements StreamGraphFactory {
         flatMap(new MyStatsCounter()).
         sendTo(pageViewPerMemberCounters);
 
-    return graph;
   }
 
   // standalone local program model

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/test/java/org/apache/samza/example/NoContextStreamExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/NoContextStreamExample.java b/samza-operator/src/test/java/org/apache/samza/example/NoContextStreamExample.java
index 577d06f..c6d2e6e 100644
--- a/samza-operator/src/test/java/org/apache/samza/example/NoContextStreamExample.java
+++ b/samza-operator/src/test/java/org/apache/samza/example/NoContextStreamExample.java
@@ -18,12 +18,9 @@
  */
 package org.apache.samza.example;
 
-import org.apache.samza.operators.OutputStream;
-import org.apache.samza.operators.StreamGraphFactory;
+import org.apache.samza.operators.*;
+import org.apache.samza.operators.StreamGraphBuilder;
 import org.apache.samza.config.Config;
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.StreamGraph;
-import org.apache.samza.operators.StreamSpec;
 import org.apache.samza.operators.data.InputMessageEnvelope;
 import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
 import org.apache.samza.operators.data.Offset;
@@ -41,9 +38,9 @@ import java.util.Properties;
 
 
 /**
- * Example {@link StreamGraphFactory} code to test the API methods
+ * Example {@link StreamGraphBuilder} code to test the API methods
  */
-public class NoContextStreamExample implements StreamGraphFactory {
+public class NoContextStreamExample implements StreamGraphBuilder {
 
   StreamSpec input1 = new StreamSpec() {
     @Override public SystemStream getSystemStream() {
@@ -129,8 +126,7 @@ public class NoContextStreamExample implements StreamGraphFactory {
    *   }
    *
    */
-  @Override public StreamGraph create(Config config) {
-    StreamGraph graph = StreamGraph.fromConfig(config);
+  @Override public void init(StreamGraph graph, Config config) {
     MessageStream<InputMessageEnvelope> inputSource1 = graph.<Object, Object, InputMessageEnvelope>createInStream(
         input1, null, null);
     MessageStream<InputMessageEnvelope> inputSource2 = graph.<Object, Object, InputMessageEnvelope>createInStream(
@@ -142,7 +138,6 @@ public class NoContextStreamExample implements StreamGraphFactory {
         join(inputSource2.map(this::getInputMessage), new MyJoinFunction()).
         sendTo(outStream);
 
-    return graph;
   }
 
   // standalone local program model

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java b/samza-operator/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java
index ad433b6..0477066 100644
--- a/samza-operator/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java
+++ b/samza-operator/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java
@@ -20,7 +20,7 @@ package org.apache.samza.example;
 
 import org.apache.samza.operators.MessageStream;
 import org.apache.samza.operators.OutputStream;
-import org.apache.samza.operators.StreamGraphFactory;
+import org.apache.samza.operators.StreamGraphBuilder;
 import org.apache.samza.config.Config;
 import org.apache.samza.operators.StreamGraph;
 import org.apache.samza.operators.StreamSpec;
@@ -38,7 +38,7 @@ import java.util.Properties;
 /**
  * Simple 2-way stream-to-stream join example
  */
-public class OrderShipmentJoinExample implements StreamGraphFactory {
+public class OrderShipmentJoinExample implements StreamGraphBuilder {
 
   /**
    * used by remote execution environment to launch the job in remote program. The remote program should follow the similar
@@ -53,8 +53,7 @@ public class OrderShipmentJoinExample implements StreamGraphFactory {
    *   }
    *
    */
-  @Override public StreamGraph create(Config config) {
-    StreamGraph graph = StreamGraph.fromConfig(config);
+  @Override public void init(StreamGraph graph, Config config) {
 
     MessageStream<OrderRecord> orders = graph.createInStream(input1, new StringSerde("UTF-8"), new JsonSerde<>());
     MessageStream<ShipmentRecord> shipments = graph.createInStream(input2, new StringSerde("UTF-8"), new JsonSerde<>());
@@ -62,7 +61,6 @@ public class OrderShipmentJoinExample implements StreamGraphFactory {
 
     orders.join(shipments, new MyJoinFunction()).sendTo(fulfilledOrders);
 
-    return graph;
   }
 
   // standalone local program model

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/test/java/org/apache/samza/example/PageViewCounterExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/PageViewCounterExample.java b/samza-operator/src/test/java/org/apache/samza/example/PageViewCounterExample.java
index 1502aa2..f7d8bda 100644
--- a/samza-operator/src/test/java/org/apache/samza/example/PageViewCounterExample.java
+++ b/samza-operator/src/test/java/org/apache/samza/example/PageViewCounterExample.java
@@ -18,12 +18,9 @@
  */
 package org.apache.samza.example;
 
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.OutputStream;
-import org.apache.samza.operators.StreamGraphFactory;
+import org.apache.samza.operators.*;
+import org.apache.samza.operators.StreamGraphBuilder;
 import org.apache.samza.config.Config;
-import org.apache.samza.operators.StreamGraph;
-import org.apache.samza.operators.StreamSpec;
 import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.operators.triggers.Triggers;
 import org.apache.samza.operators.windows.AccumulationMode;
@@ -42,10 +39,9 @@ import java.util.Properties;
 /**
  * Example code to implement window-based counter
  */
-public class PageViewCounterExample implements StreamGraphFactory {
+public class PageViewCounterExample implements StreamGraphBuilder {
 
-  @Override public StreamGraph create(Config config) {
-    StreamGraph graph = StreamGraph.fromConfig(config);
+  @Override public void init(StreamGraph graph, Config config) {
 
     MessageStream<PageViewEvent> pageViewEvents = graph.createInStream(input1, new StringSerde("UTF-8"), new JsonSerde<>());
     OutputStream<MyStreamOutput> pageViewPerMemberCounters = graph.createOutStream(output, new StringSerde("UTF-8"), new JsonSerde<>());
@@ -56,7 +52,7 @@ public class PageViewCounterExample implements StreamGraphFactory {
             setAccumulationMode(AccumulationMode.DISCARDING)).
         map(MyStreamOutput::new).
         sendTo(pageViewPerMemberCounters);
-    return graph;
+
   }
 
   public static void main(String[] args) {

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/test/java/org/apache/samza/example/RepartitionExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/RepartitionExample.java b/samza-operator/src/test/java/org/apache/samza/example/RepartitionExample.java
index f15e514..6994ac4 100644
--- a/samza-operator/src/test/java/org/apache/samza/example/RepartitionExample.java
+++ b/samza-operator/src/test/java/org/apache/samza/example/RepartitionExample.java
@@ -18,12 +18,9 @@
  */
 package org.apache.samza.example;
 
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.OutputStream;
-import org.apache.samza.operators.StreamGraphFactory;
+import org.apache.samza.operators.*;
+import org.apache.samza.operators.StreamGraphBuilder;
 import org.apache.samza.config.Config;
-import org.apache.samza.operators.StreamGraph;
-import org.apache.samza.operators.StreamSpec;
 import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.operators.windows.WindowPane;
 import org.apache.samza.operators.windows.Windows;
@@ -38,9 +35,9 @@ import java.util.*;
 
 
 /**
- * Example {@link StreamGraphFactory} code to test the API methods with re-partition operator
+ * Example {@link StreamGraphBuilder} code to test the API methods with re-partition operator
  */
-public class RepartitionExample implements StreamGraphFactory {
+public class RepartitionExample implements StreamGraphBuilder {
 
   /**
    * used by remote execution environment to launch the job in remote program. The remote program should follow the similar
@@ -54,8 +51,7 @@ public class RepartitionExample implements StreamGraphFactory {
    *   }
    *
    */
-  @Override public StreamGraph create(Config config) {
-    StreamGraph graph = StreamGraph.fromConfig(config);
+  @Override public void init(StreamGraph graph, Config config) {
 
     MessageStream<PageViewEvent> pageViewEvents = graph.createInStream(input1, new StringSerde("UTF-8"), new JsonSerde<>());
     OutputStream<MyStreamOutput> pageViewPerMemberCounters = graph.createOutStream(output, new StringSerde("UTF-8"), new JsonSerde<>());
@@ -67,7 +63,6 @@ public class RepartitionExample implements StreamGraphFactory {
         map(MyStreamOutput::new).
         sendTo(pageViewPerMemberCounters);
 
-    return graph;
   }
 
   // standalone local program model

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/test/java/org/apache/samza/example/TestBasicStreamGraphs.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/TestBasicStreamGraphs.java b/samza-operator/src/test/java/org/apache/samza/example/TestBasicStreamGraphs.java
new file mode 100644
index 0000000..8ecd44f
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/TestBasicStreamGraphs.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.lang.reflect.Field;
+import org.apache.samza.Partition;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.impl.OperatorGraph;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.task.StreamOperatorTask;
+import org.apache.samza.task.TaskContext;
+import org.junit.Test;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+
+/**
+ * Unit test for {@link StreamOperatorTask}
+ */
+public class TestBasicStreamGraphs {
+
+  private final Set<SystemStreamPartition> inputPartitions = new HashSet<SystemStreamPartition>() { {
+      for (int i = 0; i < 4; i++) {
+        this.add(new SystemStreamPartition("my-system", String.format("my-topic%d", i), new Partition(i)));
+      }
+    } };
+
+  @Test
+  public void testUserTask() throws Exception {
+    Config mockConfig = mock(Config.class);
+    TaskContext mockContext = mock(TaskContext.class);
+    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
+    TestWindowExample userTask = new TestWindowExample(this.inputPartitions);
+    StreamOperatorTask adaptorTask = new StreamOperatorTask(userTask);
+    Field pipelineMapFld = StreamOperatorTask.class.getDeclaredField("operatorGraph");
+    pipelineMapFld.setAccessible(true);
+    OperatorGraph opGraph = (OperatorGraph) pipelineMapFld.get(adaptorTask);
+
+    adaptorTask.init(mockConfig, mockContext);
+    this.inputPartitions.forEach(partition -> {
+        assertNotNull(opGraph.get(partition.getSystemStream()));
+      });
+  }
+
+  @Test
+  public void testSplitTask() throws Exception {
+    Config mockConfig = mock(Config.class);
+    TaskContext mockContext = mock(TaskContext.class);
+    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
+    TestBroadcastExample splitTask = new TestBroadcastExample(this.inputPartitions);
+    StreamOperatorTask adaptorTask = new StreamOperatorTask(splitTask);
+    Field pipelineMapFld = StreamOperatorTask.class.getDeclaredField("operatorGraph");
+    pipelineMapFld.setAccessible(true);
+    OperatorGraph opGraph = (OperatorGraph) pipelineMapFld.get(adaptorTask);
+
+    adaptorTask.init(mockConfig, mockContext);
+    this.inputPartitions.forEach(partition -> {
+        assertNotNull(opGraph.get(partition.getSystemStream()));
+      });
+  }
+
+  @Test
+  public void testJoinTask() throws Exception {
+    Config mockConfig = mock(Config.class);
+    TaskContext mockContext = mock(TaskContext.class);
+    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
+    TestJoinExample joinTask = new TestJoinExample(this.inputPartitions);
+    StreamOperatorTask adaptorTask = new StreamOperatorTask(joinTask);
+    Field pipelineMapFld = StreamOperatorTask.class.getDeclaredField("operatorGraph");
+    pipelineMapFld.setAccessible(true);
+    OperatorGraph opGraph = (OperatorGraph) pipelineMapFld.get(adaptorTask);
+
+    adaptorTask.init(mockConfig, mockContext);
+    this.inputPartitions.forEach(partition -> {
+        assertNotNull(opGraph.get(partition.getSystemStream()));
+      });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/test/java/org/apache/samza/example/TestBroadcastExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/TestBroadcastExample.java b/samza-operator/src/test/java/org/apache/samza/example/TestBroadcastExample.java
new file mode 100644
index 0000000..d22324b
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/TestBroadcastExample.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.example;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamSpec;
+import org.apache.samza.operators.data.InputMessageEnvelope;
+import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.Offset;
+import org.apache.samza.operators.triggers.Triggers;
+import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.time.Duration;
+import java.util.function.BiFunction;
+import java.util.Properties;
+import java.util.Set;
+
+
+/**
+ * Example implementation of split stream tasks
+ *
+ */
+public class TestBroadcastExample extends TestExampleBase {
+
+  TestBroadcastExample(Set<SystemStreamPartition> inputs) {
+    super(inputs);
+  }
+
+  class MessageType {
+    String field1;
+    String field2;
+    String field3;
+    String field4;
+    String parKey;
+    private long timestamp;
+
+    public long getTimestamp() {
+      return this.timestamp;
+    }
+  }
+
+  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
+    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
+      super(key, data, offset, partition);
+    }
+  }
+
+  @Override
+  public void init(StreamGraph graph, Config config) {
+    BiFunction<JsonMessageEnvelope, Integer, Integer> sumAggregator = (m, c) -> c + 1;
+    inputs.keySet().forEach(entry -> {
+        MessageStream<JsonMessageEnvelope> inputStream = graph.<Object, Object, InputMessageEnvelope>createInStream(new StreamSpec() {
+          @Override public SystemStream getSystemStream() {
+            return entry;
+          }
+
+          @Override public Properties getProperties() {
+            return null;
+          }
+        }, null, null).map(this::getInputMessage);
+
+        inputStream.filter(this::myFilter1).window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
+            .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
+
+        inputStream.filter(this::myFilter2).window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
+            .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
+
+        inputStream.filter(this::myFilter3).window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
+            .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
+
+      });
+  }
+
+  JsonMessageEnvelope getInputMessage(InputMessageEnvelope m1) {
+    return (JsonMessageEnvelope) m1.getMessage();
+  }
+
+  boolean myFilter1(JsonMessageEnvelope m1) {
+    // Do user defined processing here
+    return m1.getMessage().parKey.equals("key1");
+  }
+
+  boolean myFilter2(JsonMessageEnvelope m1) {
+    // Do user defined processing here
+    return m1.getMessage().parKey.equals("key2");
+  }
+
+  boolean myFilter3(JsonMessageEnvelope m1) {
+    return m1.getMessage().parKey.equals("key3");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/test/java/org/apache/samza/example/TestExampleBase.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/TestExampleBase.java b/samza-operator/src/test/java/org/apache/samza/example/TestExampleBase.java
new file mode 100644
index 0000000..c4df9d4
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/TestExampleBase.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.example;
+
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Base class for test examples
+ *
+ */
+public abstract class TestExampleBase implements StreamGraphBuilder {
+
+  protected final Map<SystemStream, Set<SystemStreamPartition>> inputs;
+
+  TestExampleBase(Set<SystemStreamPartition> inputs) {
+    this.inputs = new HashMap<>();
+    for (SystemStreamPartition input : inputs) {
+      this.inputs.putIfAbsent(input.getSystemStream(), new HashSet<>());
+      this.inputs.get(input.getSystemStream()).add(input);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/test/java/org/apache/samza/example/TestFluentStreamTasks.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/TestFluentStreamTasks.java b/samza-operator/src/test/java/org/apache/samza/example/TestFluentStreamTasks.java
deleted file mode 100644
index 5f659ba..0000000
--- a/samza-operator/src/test/java/org/apache/samza/example/TestFluentStreamTasks.java
+++ /dev/null
@@ -1,99 +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 java.lang.reflect.Field;
-import org.apache.samza.Partition;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.impl.OperatorGraph;
-import org.apache.samza.system.SystemStreamPartition;
-import org.apache.samza.task.StreamOperatorTask;
-import org.apache.samza.task.TaskContext;
-import org.junit.Test;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
-
-
-/**
- * Unit test for {@link StreamOperatorTask}
- */
-public class TestFluentStreamTasks {
-
-  private final Set<SystemStreamPartition> inputPartitions = new HashSet<SystemStreamPartition>() { {
-      for (int i = 0; i < 4; i++) {
-        this.add(new SystemStreamPartition("my-system", String.format("my-topic%d", i), new Partition(i)));
-      }
-    } };
-
-  @Test
-  public void testUserTask() throws Exception {
-    Config mockConfig = mock(Config.class);
-    TaskContext mockContext = mock(TaskContext.class);
-    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
-    WindowGraph userTask = new WindowGraph(this.inputPartitions);
-    StreamOperatorTask adaptorTask = new StreamOperatorTask(userTask);
-    Field pipelineMapFld = StreamOperatorTask.class.getDeclaredField("operatorGraph");
-    pipelineMapFld.setAccessible(true);
-    OperatorGraph opGraph = (OperatorGraph) pipelineMapFld.get(adaptorTask);
-
-    adaptorTask.init(mockConfig, mockContext);
-    this.inputPartitions.forEach(partition -> {
-        assertNotNull(opGraph.get(partition.getSystemStream()));
-      });
-  }
-
-  @Test
-  public void testSplitTask() throws Exception {
-    Config mockConfig = mock(Config.class);
-    TaskContext mockContext = mock(TaskContext.class);
-    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
-    BroadcastGraph splitTask = new BroadcastGraph(this.inputPartitions);
-    StreamOperatorTask adaptorTask = new StreamOperatorTask(splitTask);
-    Field pipelineMapFld = StreamOperatorTask.class.getDeclaredField("operatorGraph");
-    pipelineMapFld.setAccessible(true);
-    OperatorGraph opGraph = (OperatorGraph) pipelineMapFld.get(adaptorTask);
-
-    adaptorTask.init(mockConfig, mockContext);
-    this.inputPartitions.forEach(partition -> {
-        assertNotNull(opGraph.get(partition.getSystemStream()));
-      });
-  }
-
-  @Test
-  public void testJoinTask() throws Exception {
-    Config mockConfig = mock(Config.class);
-    TaskContext mockContext = mock(TaskContext.class);
-    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
-    JoinGraph joinTask = new JoinGraph(this.inputPartitions);
-    StreamOperatorTask adaptorTask = new StreamOperatorTask(joinTask);
-    Field pipelineMapFld = StreamOperatorTask.class.getDeclaredField("operatorGraph");
-    pipelineMapFld.setAccessible(true);
-    OperatorGraph opGraph = (OperatorGraph) pipelineMapFld.get(adaptorTask);
-
-    adaptorTask.init(mockConfig, mockContext);
-    this.inputPartitions.forEach(partition -> {
-        assertNotNull(opGraph.get(partition.getSystemStream()));
-      });
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/test/java/org/apache/samza/example/TestJoinExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/TestJoinExample.java b/samza-operator/src/test/java/org/apache/samza/example/TestJoinExample.java
new file mode 100644
index 0000000..fe6e7e7
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/TestJoinExample.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.example;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamSpec;
+import org.apache.samza.operators.data.InputMessageEnvelope;
+import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.Offset;
+import org.apache.samza.operators.functions.JoinFunction;
+import org.apache.samza.serializers.JsonSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+
+
+/**
+ * Example implementation of unique key-based stream-stream join tasks
+ *
+ */
+public class TestJoinExample  extends TestExampleBase {
+
+  TestJoinExample(Set<SystemStreamPartition> inputs) {
+    super(inputs);
+  }
+
+  class MessageType {
+    String joinKey;
+    List<String> joinFields = new ArrayList<>();
+  }
+
+  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
+    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
+      super(key, data, offset, partition);
+    }
+  }
+
+  MessageStream<JsonMessageEnvelope> joinOutput = null;
+
+  @Override
+  public void init(StreamGraph graph, Config config) {
+
+    for (SystemStream input : inputs.keySet()) {
+      MessageStream<JsonMessageEnvelope> newSource = graph.<Object, Object, InputMessageEnvelope>createInStream(
+          new StreamSpec() {
+            @Override public SystemStream getSystemStream() {
+              return input;
+            }
+
+            @Override public Properties getProperties() {
+              return null;
+            }
+          }, null, null).map(this::getInputMessage);
+      if (joinOutput == null) {
+        joinOutput = newSource;
+      } else {
+        joinOutput = joinOutput.join(newSource, new MyJoinFunction());
+      }
+    }
+
+    joinOutput.sendTo(graph.createOutStream(new StreamSpec() {
+      @Override public SystemStream getSystemStream() {
+        return null;
+      }
+
+      @Override public Properties getProperties() {
+        return null;
+      }
+    }, new StringSerde("UTF-8"), new JsonSerde<>()));
+
+  }
+
+  private JsonMessageEnvelope getInputMessage(InputMessageEnvelope ism) {
+    return new JsonMessageEnvelope(
+        ((MessageType) ism.getMessage()).joinKey,
+        (MessageType) ism.getMessage(),
+        ism.getOffset(),
+        ism.getSystemStreamPartition());
+  }
+
+  class MyJoinFunction implements JoinFunction<String, JsonMessageEnvelope, JsonMessageEnvelope, JsonMessageEnvelope> {
+    JsonMessageEnvelope myJoinResult(JsonMessageEnvelope m1, JsonMessageEnvelope m2) {
+      MessageType newJoinMsg = new MessageType();
+      newJoinMsg.joinKey = m1.getKey();
+      newJoinMsg.joinFields.addAll(m1.getMessage().joinFields);
+      newJoinMsg.joinFields.addAll(m2.getMessage().joinFields);
+      return new JsonMessageEnvelope(m1.getMessage().joinKey, newJoinMsg, null, null);
+    }
+
+    @Override
+    public JsonMessageEnvelope apply(JsonMessageEnvelope message, JsonMessageEnvelope otherMessage) {
+      return this.myJoinResult(message, otherMessage);
+    }
+
+    @Override
+    public String getFirstKey(JsonMessageEnvelope message) {
+      return message.getKey();
+    }
+
+    @Override
+    public String getSecondKey(JsonMessageEnvelope message) {
+      return message.getKey();
+    }
+  }
+}


[14/14] samza git commit: Merge branch 'samza-fluent-api-v1' of http://git-wip-us.apache.org/repos/asf/samza into samza-fluent-api-v1

Posted by ni...@apache.org.
Merge branch 'samza-fluent-api-v1' of http://git-wip-us.apache.org/repos/asf/samza into samza-fluent-api-v1


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

Branch: refs/heads/samza-fluent-api-v1
Commit: 5d416cfc437f986302cfedf15c85a474c012d7b2
Parents: 55d7988 001be63
Author: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Authored: Thu Feb 16 10:39:44 2017 -0800
Committer: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Committed: Thu Feb 16 10:39:44 2017 -0800

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

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



[05/14] samza git commit: SAMZA-1073: top-level fluent API `

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
index abed03f..41d1778 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
@@ -18,21 +18,22 @@
  */
 package org.apache.samza.operators.impl;
 
-import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.config.Config;
 import org.apache.samza.operators.functions.SinkFunction;
 import org.apache.samza.operators.spec.SinkOperatorSpec;
 import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
 
 
 /**
  * Implementation for {@link SinkOperatorSpec}
  */
-class SinkOperatorImpl<M extends MessageEnvelope> extends OperatorImpl<M, MessageEnvelope> {
+class SinkOperatorImpl<M> extends OperatorImpl<M, M> {
 
   private final SinkFunction<M> sinkFn;
 
-  SinkOperatorImpl(SinkOperatorSpec<M> sinkOp) {
+  SinkOperatorImpl(SinkOperatorSpec<M> sinkOp, Config config, TaskContext context) {
     this.sinkFn = sinkOp.getSinkFn();
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java
index 3a5c56e..644de20 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java
@@ -18,24 +18,26 @@
  */
 package org.apache.samza.operators.impl;
 
-import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
 import org.apache.samza.operators.functions.FlatMapFunction;
 import org.apache.samza.operators.spec.StreamOperatorSpec;
 import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
 
 
 /**
- * A StreamOperator that accepts a 1:n transform function and applies it to each incoming {@link MessageEnvelope}.
+ * A StreamOperator that accepts a 1:n transform function and applies it to each incoming message.
  *
- * @param <M>  type of {@link MessageEnvelope} in the input stream
- * @param <RM>  type of {@link MessageEnvelope} in the output stream
+ * @param <M>  type of message in the input stream
+ * @param <RM>  type of message in the output stream
  */
-class StreamOperatorImpl<M extends MessageEnvelope, RM extends MessageEnvelope> extends OperatorImpl<M, RM> {
+class StreamOperatorImpl<M, RM> extends OperatorImpl<M, RM> {
 
   private final FlatMapFunction<M, RM> transformFn;
 
-  StreamOperatorImpl(StreamOperatorSpec<M, RM> streamOperatorSpec) {
+  StreamOperatorImpl(StreamOperatorSpec<M, RM> streamOperatorSpec, MessageStreamImpl<M> source, Config config, TaskContext context) {
     this.transformFn = streamOperatorSpec.getTransformFn();
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java
index a5b71a7..af00553 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java
@@ -18,18 +18,21 @@
  */
 package org.apache.samza.operators.impl;
 
-import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
 import org.apache.samza.operators.spec.WindowOperatorSpec;
 import org.apache.samza.operators.windows.WindowPane;
 import org.apache.samza.operators.windows.internal.WindowInternal;
 import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
 
-public class WindowOperatorImpl<M extends MessageEnvelope, K, WK, WV, WM extends WindowPane<WK, WV>> extends OperatorImpl<M, WM> {
+public class WindowOperatorImpl<M, WK, WV> extends OperatorImpl<M, WindowPane<WK, WV>> {
 
-  private final WindowInternal<M, K, WV> window;
+  private final WindowInternal<M, WK, WV> window;
 
-  public WindowOperatorImpl(WindowOperatorSpec spec) {
+  public WindowOperatorImpl(WindowOperatorSpec spec, MessageStreamImpl<M> source, Config config, TaskContext context) {
+    // source, config, and context are used to initialize the window kv-store
     window = spec.getWindow();
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
index 8b75cdc..1444662 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
@@ -18,20 +18,45 @@
  */
 package org.apache.samza.operators.spec;
 
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.task.TaskContext;
 
 
 /**
  * A stateless serializable stream operator specification that holds all the information required
- * to transform the input {@link MessageStream} and produce the output {@link MessageStream}.
+ * to transform the input {@link MessageStreamImpl} and produce the output {@link MessageStreamImpl}.
+ *
+ * @param <OM>  the type of output message from the operator
  */
-public interface OperatorSpec<OM extends MessageEnvelope> {
+@InterfaceStability.Unstable
+public interface OperatorSpec<OM> {
+
+  enum OpCode {
+    MAP,
+    FLAT_MAP,
+    FILTER,
+    SINK,
+    SEND_TO,
+    JOIN,
+    WINDOW,
+    MERGE,
+    PARTITION_BY
+  }
+
 
   /**
-   * Get the output stream containing transformed {@link MessageEnvelope} produced by this operator.
-   * @return  the output stream containing transformed {@link MessageEnvelope} produced by this operator.
+   * Get the output stream containing transformed messages produced by this operator.
+   * @return  the output stream containing transformed messages produced by this operator.
    */
-  MessageStream<OM> getOutputStream();
+  MessageStreamImpl<OM> getNextStream();
 
+  /**
+   * Init method to initialize the context for this {@link OperatorSpec}. The default implementation is NO-OP.
+   *
+   * @param config  the {@link Config} object for this task
+   * @param context  the {@link TaskContext} object for this task
+   */
+  default void init(Config config, TaskContext context) { }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java
index fc25929..d626852 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java
@@ -19,16 +19,21 @@
 
 package org.apache.samza.operators.spec;
 
-import org.apache.samza.operators.data.MessageEnvelope;
+import java.util.Collection;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.OutputStream;
+import org.apache.samza.operators.StreamGraphImpl;
+import org.apache.samza.operators.functions.FilterFunction;
 import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.functions.MapFunction;
+import org.apache.samza.operators.functions.PartialJoinFunction;
 import org.apache.samza.operators.functions.SinkFunction;
-import org.apache.samza.operators.MessageStreamImpl;
 import org.apache.samza.operators.windows.WindowPane;
 import org.apache.samza.operators.windows.internal.WindowInternal;
 
 import java.util.ArrayList;
-import java.util.UUID;
-import java.util.function.BiFunction;
+import org.apache.samza.task.TaskContext;
 
 
 /**
@@ -38,80 +43,168 @@ public class OperatorSpecs {
 
   private OperatorSpecs() {}
 
-  private static String getOperatorId() {
-    // TODO: need to change the IDs to be a consistent, durable IDs that can be recovered across container and job restarts
-    return UUID.randomUUID().toString();
+  /**
+   * Creates a {@link StreamOperatorSpec} for {@link MapFunction}
+   *
+   * @param mapFn  the map function
+   * @param graph  the {@link StreamGraphImpl} object
+   * @param output  the output {@link MessageStreamImpl} object
+   * @param <M>  type of input message
+   * @param <OM>  type of output message
+   * @return  the {@link StreamOperatorSpec}
+   */
+  public static <M, OM> StreamOperatorSpec<M, OM> createMapOperatorSpec(MapFunction<M, OM> mapFn, StreamGraphImpl graph, MessageStreamImpl<OM> output) {
+    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);
+      }
+    }, output, OperatorSpec.OpCode.MAP, graph.getNextOpId());
+  }
+
+  /**
+   * Creates a {@link StreamOperatorSpec} for {@link FilterFunction}
+   *
+   * @param filterFn  the transformation function
+   * @param graph  the {@link StreamGraphImpl} object
+   * @param output  the output {@link MessageStreamImpl} object
+   * @param <M>  type of input message
+   * @return  the {@link StreamOperatorSpec}
+   */
+  public static <M> StreamOperatorSpec<M, M> createFilterOperatorSpec(FilterFunction<M> filterFn, StreamGraphImpl graph, MessageStreamImpl<M> output) {
+    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);
+      }
+    }, output, OperatorSpec.OpCode.FILTER, graph.getNextOpId());
   }
 
   /**
    * Creates a {@link StreamOperatorSpec}.
    *
    * @param transformFn  the transformation function
-   * @param <M>  type of input {@link MessageEnvelope}
-   * @param <OM>  type of output {@link MessageEnvelope}
+   * @param graph  the {@link StreamGraphImpl} object
+   * @param output  the output {@link MessageStreamImpl} object
+   * @param <M>  type of input message
+   * @param <OM>  type of output message
    * @return  the {@link StreamOperatorSpec}
    */
-  public static <M extends MessageEnvelope, OM extends MessageEnvelope> StreamOperatorSpec<M, OM> createStreamOperatorSpec(
-      FlatMapFunction<M, OM> transformFn) {
-    return new StreamOperatorSpec<>(transformFn);
+  public static <M, OM> StreamOperatorSpec<M, OM> createStreamOperatorSpec(
+      FlatMapFunction<M, OM> transformFn, StreamGraphImpl graph, MessageStreamImpl<OM> output) {
+    return new StreamOperatorSpec<>(transformFn, output, OperatorSpec.OpCode.FLAT_MAP, graph.getNextOpId());
+  }
+
+  /**
+   * Creates a {@link SinkOperatorSpec}.
+   *
+   * @param sinkFn  the sink function
+   * @param <M>  type of input message
+   * @param graph  the {@link StreamGraphImpl} object
+   * @return  the {@link SinkOperatorSpec}
+   */
+  public static <M> SinkOperatorSpec<M> createSinkOperatorSpec(SinkFunction<M> sinkFn, StreamGraphImpl graph) {
+    return new SinkOperatorSpec<>(sinkFn, OperatorSpec.OpCode.SINK, graph.getNextOpId());
+  }
+
+  /**
+   * Creates a {@link SinkOperatorSpec}.
+   *
+   * @param sinkFn  the sink function
+   * @param graph  the {@link StreamGraphImpl} object
+   * @param stream  the {@link OutputStream} where the message is sent to
+   * @param <M>  type of input message
+   * @return  the {@link SinkOperatorSpec}
+   */
+  public static <M> SinkOperatorSpec<M> createSendToOperatorSpec(SinkFunction<M> sinkFn, StreamGraphImpl graph, OutputStream<M> stream) {
+    return new SinkOperatorSpec<>(sinkFn, OperatorSpec.OpCode.SEND_TO, graph.getNextOpId(), stream);
   }
 
   /**
    * Creates a {@link SinkOperatorSpec}.
    *
    * @param sinkFn  the sink function
-   * @param <M>  type of input {@link MessageEnvelope}
+   * @param graph  the {@link StreamGraphImpl} object
+   * @param stream  the {@link OutputStream} where the message is sent to
+   * @param <M>  type of input message
    * @return  the {@link SinkOperatorSpec}
    */
-  public static <M extends MessageEnvelope> SinkOperatorSpec<M> createSinkOperatorSpec(SinkFunction<M> sinkFn) {
-    return new SinkOperatorSpec<>(sinkFn);
+  public static <M> SinkOperatorSpec<M> createPartitionOperatorSpec(SinkFunction<M> sinkFn, StreamGraphImpl graph, OutputStream<M> stream) {
+    return new SinkOperatorSpec<>(sinkFn, OperatorSpec.OpCode.PARTITION_BY, graph.getNextOpId(), stream);
   }
 
   /**
    * Creates a {@link WindowOperatorSpec}.
    *
    * @param window the description of the window.
-   * @param <M> the type of input {@link MessageEnvelope}
-   * @param <K> the type of key in the {@link MessageEnvelope} in this {@link org.apache.samza.operators.MessageStream}. If a key is specified,
-   *            results are emitted per-key
+   * @param graph  the {@link StreamGraphImpl} object
+   * @param wndOutput  the window output {@link MessageStreamImpl} object
+   * @param <M> the type of input message
    * @param <WK> the type of key in the {@link WindowPane}
    * @param <WV> the type of value in the window
-   * @param <WM> the type of output {@link WindowPane}
    * @return  the {@link WindowOperatorSpec}
    */
 
-  public static <M extends MessageEnvelope, K, WK, WV, WM extends WindowPane<WK, WV>> WindowOperatorSpec<M, K, WK, WV, WM> createWindowOperatorSpec(WindowInternal<M, K, WV> window) {
-    return new WindowOperatorSpec<>(window, OperatorSpecs.getOperatorId());
+  public static <M, WK, WV> WindowOperatorSpec<M, WK, WV> createWindowOperatorSpec(
+      WindowInternal<M, WK, WV> window, StreamGraphImpl graph, MessageStreamImpl<WindowPane<WK, WV>> wndOutput) {
+    return new WindowOperatorSpec<>(window, wndOutput, graph.getNextOpId());
   }
 
   /**
    * Creates a {@link PartialJoinOperatorSpec}.
    *
    * @param partialJoinFn  the join function
+   * @param graph  the {@link StreamGraphImpl} object
    * @param joinOutput  the output {@link MessageStreamImpl}
-   * @param <M>  type of input {@link MessageEnvelope}
+   * @param <M>  type of input message
    * @param <K>  type of join key
-   * @param <JM>  the type of {@link MessageEnvelope} in the other join stream
-   * @param <OM>  the type of {@link MessageEnvelope} in the join output
+   * @param <JM>  the type of message in the other join stream
+   * @param <OM>  the type of message in the join output
    * @return  the {@link PartialJoinOperatorSpec}
    */
-  public static <M extends MessageEnvelope<K, ?>, K, JM extends MessageEnvelope<K, ?>, OM extends MessageEnvelope> PartialJoinOperatorSpec<M, K, JM, OM> createPartialJoinOperatorSpec(
-      BiFunction<M, JM, OM> partialJoinFn, MessageStreamImpl<OM> joinOutput) {
-    return new PartialJoinOperatorSpec<>(partialJoinFn, joinOutput, OperatorSpecs.getOperatorId());
+  public static <M, K, JM, OM> PartialJoinOperatorSpec<M, K, JM, OM> createPartialJoinOperatorSpec(
+      PartialJoinFunction<K, M, JM, OM> partialJoinFn, StreamGraphImpl graph, MessageStreamImpl<OM> joinOutput) {
+    return new PartialJoinOperatorSpec<>(partialJoinFn, joinOutput, graph.getNextOpId());
   }
 
   /**
    * Creates a {@link StreamOperatorSpec} with a merger function.
    *
+   * @param graph  the {@link StreamGraphImpl} object
    * @param mergeOutput  the output {@link MessageStreamImpl} from the merger
-   * @param <M>  the type of input {@link MessageEnvelope}
+   * @param <M>  the type of input message
    * @return  the {@link StreamOperatorSpec} for the merge
    */
-  public static <M extends MessageEnvelope> StreamOperatorSpec<M, M> createMergeOperatorSpec(MessageStreamImpl<M> mergeOutput) {
-    return new StreamOperatorSpec<M, M>(t ->
-      new ArrayList<M>() { {
-          this.add(t);
-        } },
-      mergeOutput);
+  public static <M> StreamOperatorSpec<M, M> createMergeOperatorSpec(StreamGraphImpl graph, MessageStreamImpl<M> mergeOutput) {
+    return new StreamOperatorSpec<M, M>(message ->
+        new ArrayList<M>() {
+          {
+            this.add(message);
+          }
+        },
+        mergeOutput, OperatorSpec.OpCode.MERGE, graph.getNextOpId());
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java
index e6d77f6..e057c2b 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java
@@ -18,63 +18,69 @@
  */
 package org.apache.samza.operators.spec;
 
-import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.config.Config;
 import org.apache.samza.operators.MessageStreamImpl;
-
-import java.util.function.BiFunction;
+import org.apache.samza.operators.functions.PartialJoinFunction;
+import org.apache.samza.task.TaskContext;
 
 
 /**
- * Spec for the partial join operator that takes {@link MessageEnvelope}s from one input stream, joins with buffered
- * {@link MessageEnvelope}s from another stream, and produces join results to an output {@link MessageStreamImpl}.
+ * Spec for the partial join operator that takes messages from one input stream, joins with buffered
+ * messages from another stream, and produces join results to an output {@link MessageStreamImpl}.
  *
- * @param <M>  the type of input {@link MessageEnvelope}
+ * @param <M>  the type of input message
  * @param <K>  the type of join key
- * @param <JM>  the type of {@link MessageEnvelope} in the other join stream
- * @param <RM>  the type of {@link MessageEnvelope} in the join output stream
+ * @param <JM>  the type of message in the other join stream
+ * @param <RM>  the type of message in the join output stream
  */
-public class PartialJoinOperatorSpec<M extends MessageEnvelope<K, ?>, K, JM extends MessageEnvelope<K, ?>, RM extends MessageEnvelope>
-    implements OperatorSpec<RM> {
+public class PartialJoinOperatorSpec<M, K, JM, RM> implements OperatorSpec<RM> {
 
   private final MessageStreamImpl<RM> joinOutput;
 
   /**
-   * The transformation function of {@link PartialJoinOperatorSpec} that takes an input {@link MessageEnvelope} of
-   * type {@code M}, joins with a stream of buffered {@link MessageEnvelope}s of type {@code JM} from another stream,
-   * and generates a joined result {@link MessageEnvelope} of type {@code RM}.
+   * The transformation function of {@link PartialJoinOperatorSpec} that takes an input message of
+   * type {@code M}, joins with a stream of buffered messages of type {@code JM} from another stream,
+   * and generates a joined result message of type {@code RM}.
    */
-  private final BiFunction<M, JM, RM> transformFn;
+  private final PartialJoinFunction<K, M, JM, RM> transformFn;
 
 
   /**
    * The unique ID for this operator.
    */
-  private final String operatorId;
+  private final int opId;
 
   /**
    * Default constructor for a {@link PartialJoinOperatorSpec}.
    *
-   * @param partialJoinFn  partial join function that take type {@code M} of input {@link MessageEnvelope} and join
-   *                       w/ type {@code JM} of buffered {@link MessageEnvelope} from another stream
+   * @param partialJoinFn  partial join function that take type {@code M} of input message and join
+   *                       w/ type {@code JM} of buffered message from another stream
    * @param joinOutput  the output {@link MessageStreamImpl} of the join results
    */
-  PartialJoinOperatorSpec(BiFunction<M, JM, RM> partialJoinFn, MessageStreamImpl<RM> joinOutput, String operatorId) {
+  PartialJoinOperatorSpec(PartialJoinFunction<K, M, JM, RM> partialJoinFn, MessageStreamImpl<RM> joinOutput, int opId) {
     this.joinOutput = joinOutput;
     this.transformFn = partialJoinFn;
-    this.operatorId = operatorId;
+    this.opId = opId;
   }
 
   @Override
-  public String toString() {
-    return this.operatorId;
-  }
-
-  @Override
-  public MessageStreamImpl<RM> getOutputStream() {
+  public MessageStreamImpl<RM> getNextStream() {
     return this.joinOutput;
   }
 
-  public BiFunction<M, JM, RM> getTransformFn() {
+  public PartialJoinFunction<K, M, JM, RM> getTransformFn() {
     return this.transformFn;
   }
+
+  public OperatorSpec.OpCode getOpCode() {
+    return OpCode.JOIN;
+  }
+
+  public int getOpId() {
+    return this.opId;
+  }
+
+  @Override public void init(Config config, TaskContext context) {
+    this.transformFn.init(config, context);
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java
index 4348bc0..ba30d67 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java
@@ -18,18 +18,30 @@
  */
 package org.apache.samza.operators.spec;
 
-import org.apache.samza.operators.data.MessageEnvelope;
-import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.config.Config;
 import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.OutputStream;
+import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.task.TaskContext;
 
 
 /**
  * The spec for a sink operator that accepts user-defined logic to output a {@link MessageStreamImpl} to an external
  * system. This is a terminal operator and does allows further operator chaining.
  *
- * @param <M>  the type of input {@link MessageEnvelope}
+ * @param <M>  the type of input message
  */
-public class SinkOperatorSpec<M extends MessageEnvelope> implements OperatorSpec {
+public class SinkOperatorSpec<M> implements OperatorSpec {
+
+  /**
+   * {@link OpCode} for this {@link SinkOperatorSpec}
+   */
+  private final OperatorSpec.OpCode opCode;
+
+  /**
+   * The unique ID for this operator.
+   */
+  private final int opId;
 
   /**
    * The user-defined sink function
@@ -37,14 +49,40 @@ public class SinkOperatorSpec<M extends MessageEnvelope> implements OperatorSpec
   private final SinkFunction<M> sinkFn;
 
   /**
-   * Default constructor for a {@link SinkOperatorSpec}.
+   * Potential output stream defined by the {@link SinkFunction}
+   */
+  private final OutputStream<M> outStream;
+
+  /**
+   * Default constructor for a {@link SinkOperatorSpec} w/o an output stream. (e.g. output is sent to remote database)
+   *
+   * @param sinkFn  a user defined {@link SinkFunction} that will be called with the output message,
+   *                the output {@link org.apache.samza.task.MessageCollector} and the
+   *                {@link org.apache.samza.task.TaskCoordinator}.
+   * @param opCode  the specific {@link OpCode} for this {@link SinkOperatorSpec}. It could be {@link OpCode#SINK}, {@link OpCode#SEND_TO},
+   *                or {@link OpCode#PARTITION_BY}
+   * @param opId  the unique id of this {@link SinkOperatorSpec} in the {@link org.apache.samza.operators.StreamGraph}
+   */
+  SinkOperatorSpec(SinkFunction<M> sinkFn, OperatorSpec.OpCode opCode, int opId) {
+    this(sinkFn, opCode, opId, null);
+  }
+
+  /**
+   * Default constructor for a {@link SinkOperatorSpec} that sends the output to an {@link OutputStream}
    *
-   * @param sinkFn  a user defined {@link SinkFunction} that will be called with the output {@link MessageEnvelope},
+   * @param sinkFn  a user defined {@link SinkFunction} that will be called with the output message,
    *                the output {@link org.apache.samza.task.MessageCollector} and the
    *                {@link org.apache.samza.task.TaskCoordinator}.
+   * @param opCode  the specific {@link OpCode} for this {@link SinkOperatorSpec}. It could be {@link OpCode#SINK}, {@link OpCode#SEND_TO},
+   *                or {@link OpCode#PARTITION_BY}
+   * @param opId  the unique id of this {@link SinkOperatorSpec} in the {@link org.apache.samza.operators.StreamGraph}
+   * @param opId  the {@link OutputStream} for this {@link SinkOperatorSpec}
    */
-  SinkOperatorSpec(SinkFunction<M> sinkFn) {
+  SinkOperatorSpec(SinkFunction<M> sinkFn, OperatorSpec.OpCode opCode, int opId, OutputStream<M> outStream) {
     this.sinkFn = sinkFn;
+    this.opCode = opCode;
+    this.opId = opId;
+    this.outStream = outStream;
   }
 
   /**
@@ -52,11 +90,27 @@ public class SinkOperatorSpec<M extends MessageEnvelope> implements OperatorSpec
    * @return  null
    */
   @Override
-  public MessageStreamImpl getOutputStream() {
+  public MessageStreamImpl<M> getNextStream() {
     return null;
   }
 
   public SinkFunction<M> getSinkFn() {
     return this.sinkFn;
   }
+
+  public OperatorSpec.OpCode getOpCode() {
+    return this.opCode;
+  }
+
+  public int getOpId() {
+    return this.opId;
+  }
+
+  public OutputStream<M> getOutStream() {
+    return this.outStream;
+  }
+
+  @Override public void init(Config config, TaskContext context) {
+    this.sinkFn.init(config, context);
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java
index ed18da4..d7813f7 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java
@@ -18,50 +18,74 @@
  */
 package org.apache.samza.operators.spec;
 
-import org.apache.samza.operators.data.MessageEnvelope;
-import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.config.Config;
 import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.task.TaskContext;
 
 
 /**
- * The spec for a linear stream operator that outputs 0 or more {@link MessageEnvelope}s for each input {@link MessageEnvelope}.
+ * The spec for a linear stream operator that outputs 0 or more messages for each input message.
  *
- * @param <M>  the type of input {@link MessageEnvelope}
- * @param <OM>  the type of output {@link MessageEnvelope}
+ * @param <M>  the type of input message
+ * @param <OM>  the type of output message
  */
-public class StreamOperatorSpec<M extends MessageEnvelope, OM extends MessageEnvelope> implements OperatorSpec<OM> {
+public class StreamOperatorSpec<M, OM> implements OperatorSpec<OM> {
 
-  private final MessageStreamImpl<OM> outputStream;
+  /**
+   * {@link OpCode} for this {@link StreamOperatorSpec}
+   */
+  private final OperatorSpec.OpCode opCode;
 
-  private final FlatMapFunction<M, OM> transformFn;
+  /**
+   * The unique ID for this operator.
+   */
+  private final int opId;
 
   /**
-   * Default constructor for a {@link StreamOperatorSpec}.
-   *
-   * @param transformFn  the transformation function that transforms each input {@link MessageEnvelope} into a collection
-   *                     of output {@link MessageEnvelope}s
+   * The output {@link MessageStreamImpl} from this {@link StreamOperatorSpec}
    */
-  StreamOperatorSpec(FlatMapFunction<M, OM> transformFn) {
-    this(transformFn, new MessageStreamImpl<>());
-  }
+  private final MessageStreamImpl<OM> outputStream;
+
+  /**
+   * Transformation function applied in this {@link StreamOperatorSpec}
+   */
+  private final FlatMapFunction<M, OM> transformFn;
 
   /**
    * Constructor for a {@link StreamOperatorSpec} that accepts an output {@link MessageStreamImpl}.
    *
    * @param transformFn  the transformation function
    * @param outputStream  the output {@link MessageStreamImpl}
+   * @param opCode  the {@link OpCode} for this {@link StreamOperatorSpec}
+   * @param opId  the unique id for this {@link StreamOperatorSpec} in a {@link org.apache.samza.operators.StreamGraph}
    */
-  StreamOperatorSpec(FlatMapFunction<M, OM> transformFn, MessageStreamImpl<OM> outputStream) {
+  StreamOperatorSpec(FlatMapFunction<M, OM> transformFn, MessageStreamImpl outputStream, OperatorSpec.OpCode opCode, int opId) {
     this.outputStream = outputStream;
     this.transformFn = transformFn;
+    this.opCode = opCode;
+    this.opId = opId;
   }
 
   @Override
-  public MessageStreamImpl<OM> getOutputStream() {
+  public MessageStreamImpl<OM> getNextStream() {
     return this.outputStream;
   }
 
   public FlatMapFunction<M, OM> getTransformFn() {
     return this.transformFn;
   }
+
+  public OperatorSpec.OpCode getOpCode() {
+    return this.opCode;
+  }
+
+  public int getOpId() {
+    return this.opId;
+  }
+
+  @Override
+  public void init(Config config, TaskContext context) {
+    this.transformFn.init(config, context);
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java
index cdc02a8..46417ed 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java
@@ -19,29 +19,42 @@
 
 package org.apache.samza.operators.spec;
 
-import org.apache.samza.operators.MessageStream;
 import org.apache.samza.operators.MessageStreamImpl;
-import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.operators.windows.WindowPane;
 import org.apache.samza.operators.windows.internal.WindowInternal;
 
-public class WindowOperatorSpec<M extends MessageEnvelope, K, WK, WV, WM extends WindowPane<WK, WV>> implements OperatorSpec<WM> {
 
-  private final WindowInternal window;
+/**
+ * Default window operator spec object
+ *
+ * @param <M>  the type of input message to the window
+ * @param <WK>  the type of key of the window
+ * @param <WV>  the type of aggregated value in the window output {@link WindowPane}
+ */
+public class WindowOperatorSpec<M, WK, WV> implements OperatorSpec<WindowPane<WK, WV>> {
+
+  private final WindowInternal<M, WK, WV> window;
 
-  private final MessageStreamImpl<WM> outputStream;
+  private final MessageStreamImpl<WindowPane<WK, WV>> outputStream;
 
-  private final String operatorId;
+  private final int opId;
 
 
-  public WindowOperatorSpec(WindowInternal window, String operatorId) {
+  /**
+   * Constructor for {@link WindowOperatorSpec}.
+   *
+   * @param window  the window function
+   * @param outputStream  the output {@link MessageStreamImpl} from this {@link WindowOperatorSpec}
+   * @param opId  auto-generated unique ID of this operator
+   */
+  WindowOperatorSpec(WindowInternal<M, WK, WV> window, MessageStreamImpl<WindowPane<WK, WV>> outputStream, int opId) {
+    this.outputStream = outputStream;
     this.window = window;
-    this.outputStream = new MessageStreamImpl<>();
-    this.operatorId = operatorId;
+    this.opId = opId;
   }
 
   @Override
-  public MessageStream<WM> getOutputStream() {
+  public MessageStreamImpl<WindowPane<WK, WV>> getNextStream() {
     return this.outputStream;
   }
 
@@ -49,7 +62,11 @@ public class WindowOperatorSpec<M extends MessageEnvelope, K, WK, WV, WM extends
     return window;
   }
 
-  public String getOperatorId() {
-    return operatorId;
+  public OpCode getOpCode() {
+    return OpCode.WINDOW;
+  }
+
+  public int getOpId() {
+    return this.opId;
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowState.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowState.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowState.java
index e9af043..53bca2e 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowState.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowState.java
@@ -30,20 +30,16 @@ import org.apache.samza.annotation.InterfaceStability;
 @InterfaceStability.Unstable
 public interface WindowState<WV> {
   /**
-   * Method to get the system time when the first {@link org.apache.samza.operators.data.MessageEnvelope}
-   * in the window is received
+   * Method to get the system time when the first message in the window is received
    *
-   * @return  nano-second of system time for the first {@link org.apache.samza.operators.data.MessageEnvelope}
-   *          received in the window
+   * @return  nano-second of system time for the first message received in the window
    */
   long getFirstMessageTimeNs();
 
   /**
-   * Method to get the system time when the last {@link org.apache.samza.operators.data.MessageEnvelope}
-   * in the window is received
+   * Method to get the system time when the last message in the window is received
    *
-   * @return  nano-second of system time for the last {@link org.apache.samza.operators.data.MessageEnvelope}
-   *          received in the window
+   * @return  nano-second of system time for the last message received in the window
    */
   long getLastMessageTimeNs();
 
@@ -62,9 +58,9 @@ public interface WindowState<WV> {
   long getLatestEventTimeNs();
 
   /**
-   * Method to get the total number of {@link org.apache.samza.operators.data.MessageEnvelope}s received in the window
+   * Method to get the total number of messages received in the window
    *
-   * @return  number of {@link org.apache.samza.operators.data.MessageEnvelope}s in the window
+   * @return  number of messages in the window
    */
   long getNumberMessages();
 

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/system/SingleJobExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/system/SingleJobExecutionEnvironment.java b/samza-operator/src/main/java/org/apache/samza/system/SingleJobExecutionEnvironment.java
new file mode 100644
index 0000000..60a4c60
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/system/SingleJobExecutionEnvironment.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system;
+
+import org.apache.samza.operators.StreamGraphFactory;
+import org.apache.samza.config.Config;
+
+/**
+ * This class implements the {@link ExecutionEnvironment} that runs the applications in YARN environment
+ */
+public class SingleJobExecutionEnvironment implements ExecutionEnvironment {
+
+  @Override public void run(StreamGraphFactory app, Config config) {
+    // TODO: add description of ProcessContext that is going to create a sub-DAG of the {@code graph}
+    // TODO: actually instantiate the tasks and run the job, i.e.
+    // 1. create all input/output/intermediate topics
+    // 2. create the single job configuration
+    // 3. execute JobRunner to submit the single job for the whole graph
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java b/samza-operator/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java
new file mode 100644
index 0000000..f60ff82
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.system;
+
+import org.apache.samza.operators.StreamGraphFactory;
+import org.apache.samza.config.Config;
+
+
+/**
+ * This class implements the {@link ExecutionEnvironment} that runs the applications in standalone environment
+ */
+public class StandaloneExecutionEnvironment implements ExecutionEnvironment {
+
+  @Override public void run(StreamGraphFactory app, Config config) {
+    // 1. get logic graph for optimization
+    // StreamGraph logicGraph = app.create(config);
+    // 2. potential optimization....
+    // 3. create new instance of StreamGraphFactory that would generate the optimized graph
+    // 4. create all input/output/intermediate topics
+    // 5. create the configuration for StreamProcessor
+    // 6. start the StreamProcessor w/ optimized instance of StreamGraphFactory
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java b/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java
new file mode 100644
index 0000000..fa7ec5e
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.task;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.ContextManager;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.StreamGraphFactory;
+import org.apache.samza.operators.StreamGraphImpl;
+import org.apache.samza.operators.data.InputMessageEnvelope;
+import org.apache.samza.operators.impl.OperatorGraph;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.SystemStream;
+
+import java.util.HashMap;
+import java.util.Map;
+
+
+/**
+ * Execution of the logic sub-DAG
+ *
+ *
+ * An {@link StreamTask} implementation that receives {@link InputMessageEnvelope}s and propagates them
+ * through the user's stream transformations defined in {@link StreamGraphImpl} using the
+ * {@link org.apache.samza.operators.MessageStream} APIs.
+ * <p>
+ * This class brings all the operator API implementation components together and feeds the
+ * {@link InputMessageEnvelope}s into the transformation chains.
+ * <p>
+ * It accepts an instance of the user implemented factory {@link StreamGraphFactory} as input parameter of the constructor.
+ * When its own {@link #init(Config, TaskContext)} method is called during startup, it instantiate a user-defined {@link StreamGraphImpl}
+ * from the {@link StreamGraphFactory}, calls {@link StreamGraphImpl#getContextManager()} to initialize the task-wide context
+ * for the graph, and creates a {@link MessageStreamImpl} corresponding to each of its input
+ * {@link org.apache.samza.system.SystemStreamPartition}s. Each input {@link MessageStreamImpl}
+ * will be corresponding to either an input stream or intermediate stream in {@link StreamGraphImpl}.
+ * <p>
+ * Then, this task calls {@link org.apache.samza.operators.impl.OperatorGraph#init(Map, Config, TaskContext)} for each of the input
+ * {@link MessageStreamImpl}. This instantiates the {@link org.apache.samza.operators.impl.OperatorImpl} DAG
+ * corresponding to the aforementioned {@link org.apache.samza.operators.spec.OperatorSpec} DAG and returns the
+ * root node of the DAG, which this class saves.
+ * <p>
+ * Now that it has the root for the DAG corresponding to each {@link org.apache.samza.system.SystemStreamPartition}, it
+ * can pass the message envelopes received in {@link StreamTask#process(IncomingMessageEnvelope, MessageCollector, TaskCoordinator)}
+ * along to the appropriate root nodes. From then on, each {@link org.apache.samza.operators.impl.OperatorImpl} propagates
+ * its transformed output to the next set of {@link org.apache.samza.operators.impl.OperatorImpl}s.
+ */
+public final class StreamOperatorTask implements StreamTask, InitableTask, WindowableTask, ClosableTask {
+
+  /**
+   * A mapping from each {@link SystemStream} to the root node of its operator chain DAG.
+   */
+  private final OperatorGraph operatorGraph = new OperatorGraph();
+
+  private final StreamGraphFactory graphFactory;
+
+  private ContextManager taskManager;
+
+  public StreamOperatorTask(StreamGraphFactory graphFactory) {
+    this.graphFactory = graphFactory;
+  }
+
+  @Override
+  public final void init(Config config, TaskContext context) throws Exception {
+    // create the MessageStreamsImpl object and initialize app-specific logic DAG within the task
+    StreamGraphImpl streams = (StreamGraphImpl) this.graphFactory.create(config);
+    this.taskManager = streams.getContextManager();
+
+    Map<SystemStream, MessageStreamImpl> inputBySystemStream = new HashMap<>();
+    context.getSystemStreamPartitions().forEach(ssp -> {
+        if (!inputBySystemStream.containsKey(ssp.getSystemStream())) {
+          inputBySystemStream.putIfAbsent(ssp.getSystemStream(), streams.getInputStream(ssp.getSystemStream()));
+        }
+      });
+    operatorGraph.init(inputBySystemStream, config, this.taskManager.initTaskContext(config, context));
+  }
+
+  @Override
+  public final void process(IncomingMessageEnvelope ime, MessageCollector collector, TaskCoordinator coordinator) {
+    this.operatorGraph.get(ime.getSystemStreamPartition().getSystemStream())
+        .onNext(new InputMessageEnvelope(ime), collector, coordinator);
+  }
+
+  @Override
+  public final void window(MessageCollector collector, TaskCoordinator coordinator) throws Exception {
+    // TODO: invoke timer based triggers
+  }
+
+  @Override
+  public void close() throws Exception {
+    this.taskManager.finalizeTaskContext();
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/test/java/org/apache/samza/example/BroadcastGraph.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/BroadcastGraph.java b/samza-operator/src/test/java/org/apache/samza/example/BroadcastGraph.java
new file mode 100644
index 0000000..a91ce09
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/BroadcastGraph.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.config.Config;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamGraphFactory;
+import org.apache.samza.operators.StreamGraphImpl;
+import org.apache.samza.operators.StreamSpec;
+import org.apache.samza.operators.data.InputMessageEnvelope;
+import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.Offset;
+import org.apache.samza.operators.triggers.Triggers;
+import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.time.Duration;
+import java.util.function.BiFunction;
+import java.util.Properties;
+import java.util.Set;
+
+
+/**
+ * Example implementation of split stream tasks
+ *
+ */
+public class BroadcastGraph implements StreamGraphFactory {
+
+  private final Set<SystemStreamPartition> inputs;
+
+  BroadcastGraph(Set<SystemStreamPartition> inputs) {
+    this.inputs = inputs;
+  }
+
+  class MessageType {
+    String field1;
+    String field2;
+    String field3;
+    String field4;
+    String parKey;
+    private long timestamp;
+
+    public long getTimestamp() {
+      return this.timestamp;
+    }
+  }
+
+  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
+    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
+      super(key, data, offset, partition);
+    }
+  }
+
+  @Override
+  public StreamGraph create(Config config) {
+    StreamGraphImpl graph = new StreamGraphImpl();
+
+    BiFunction<JsonMessageEnvelope, Integer, Integer> sumAggregator = (m, c) -> c + 1;
+    inputs.forEach(entry -> {
+        MessageStream<JsonMessageEnvelope> inputStream = graph.<Object, Object, InputMessageEnvelope>createInStream(new StreamSpec() {
+          @Override public SystemStream getSystemStream() {
+            return entry.getSystemStream();
+          }
+
+          @Override public Properties getProperties() {
+            return null;
+          }
+        }, null, null).
+            map(this::getInputMessage);
+
+        inputStream.filter(this::myFilter1).window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
+            .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
+
+        inputStream.filter(this::myFilter2).window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
+            .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
+
+        inputStream.filter(this::myFilter3).window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
+            .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
+
+      });
+    return graph;
+  }
+
+  JsonMessageEnvelope getInputMessage(InputMessageEnvelope m1) {
+    return (JsonMessageEnvelope) m1.getMessage();
+  }
+
+  boolean myFilter1(JsonMessageEnvelope m1) {
+    // Do user defined processing here
+    return m1.getMessage().parKey.equals("key1");
+  }
+
+  boolean myFilter2(JsonMessageEnvelope m1) {
+    // Do user defined processing here
+    return m1.getMessage().parKey.equals("key2");
+  }
+
+  boolean myFilter3(JsonMessageEnvelope m1) {
+    return m1.getMessage().parKey.equals("key3");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/test/java/org/apache/samza/example/JoinGraph.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/JoinGraph.java b/samza-operator/src/test/java/org/apache/samza/example/JoinGraph.java
new file mode 100644
index 0000000..2313f63
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/JoinGraph.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.example;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamGraphFactory;
+import org.apache.samza.operators.StreamGraphImpl;
+import org.apache.samza.operators.StreamSpec;
+import org.apache.samza.operators.data.InputMessageEnvelope;
+import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.Offset;
+import org.apache.samza.operators.functions.KeyValueJoinFunction;
+import org.apache.samza.serializers.JsonSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+
+
+/**
+ * Example implementation of unique key-based stream-stream join tasks
+ *
+ */
+public class JoinGraph implements StreamGraphFactory {
+  private final Set<SystemStreamPartition> inputs;
+
+  JoinGraph(Set<SystemStreamPartition> inputs) {
+    this.inputs = inputs;
+  }
+
+  class MessageType {
+    String joinKey;
+    List<String> joinFields = new ArrayList<>();
+  }
+
+  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
+    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
+      super(key, data, offset, partition);
+    }
+  }
+
+  MessageStream<JsonMessageEnvelope> joinOutput = null;
+
+  @Override
+  public StreamGraph create(Config config) {
+    StreamGraphImpl graph = new StreamGraphImpl();
+
+    for (SystemStreamPartition input : inputs) {
+      MessageStream<JsonMessageEnvelope> newSource = graph.<Object, Object, InputMessageEnvelope>createInStream(
+          new StreamSpec() {
+            @Override public SystemStream getSystemStream() {
+              return input.getSystemStream();
+            }
+
+            @Override public Properties getProperties() {
+              return null;
+            }
+          }, null, null).map(this::getInputMessage);
+      if (joinOutput == null) {
+        joinOutput = newSource;
+      } else {
+        joinOutput = joinOutput.join(newSource,
+            (KeyValueJoinFunction<String, JsonMessageEnvelope, JsonMessageEnvelope, JsonMessageEnvelope>) this::myJoinResult);
+      }
+    }
+
+    joinOutput.sendTo(graph.createOutStream(new StreamSpec() {
+      @Override public SystemStream getSystemStream() {
+        return null;
+      }
+
+      @Override public Properties getProperties() {
+        return null;
+      }
+    }, new StringSerde("UTF-8"), new JsonSerde<>()));
+
+    return graph;
+  }
+
+  private JsonMessageEnvelope getInputMessage(InputMessageEnvelope ism) {
+    return new JsonMessageEnvelope(
+        ((MessageType) ism.getMessage()).joinKey,
+        (MessageType) ism.getMessage(),
+        ism.getOffset(),
+        ism.getSystemStreamPartition());
+  }
+
+  JsonMessageEnvelope myJoinResult(JsonMessageEnvelope m1, JsonMessageEnvelope m2) {
+    MessageType newJoinMsg = new MessageType();
+    newJoinMsg.joinKey = m1.getKey();
+    newJoinMsg.joinFields.addAll(m1.getMessage().joinFields);
+    newJoinMsg.joinFields.addAll(m2.getMessage().joinFields);
+    return new JsonMessageEnvelope(m1.getMessage().joinKey, newJoinMsg, null, null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/test/java/org/apache/samza/example/KeyValueStoreExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/KeyValueStoreExample.java b/samza-operator/src/test/java/org/apache/samza/example/KeyValueStoreExample.java
new file mode 100644
index 0000000..ad6336a
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/KeyValueStoreExample.java
@@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.OutputStream;
+import org.apache.samza.operators.StreamGraphFactory;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamSpec;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.serializers.JsonSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.storage.kv.KeyValueStore;
+import org.apache.samza.system.ExecutionEnvironment;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.util.CommandLine;
+
+import java.util.Properties;
+
+
+/**
+ * Example code using {@link KeyValueStore} to implement event-time window
+ */
+public class KeyValueStoreExample implements StreamGraphFactory {
+
+  /**
+   * used by remote execution environment to launch the job in remote program. The remote program should follow the similar
+   * invoking context as in standalone:
+   *
+   *   public static void main(String args[]) throws Exception {
+   *     CommandLine cmdLine = new CommandLine();
+   *     Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+   *     ExecutionEnvironment remoteEnv = ExecutionEnvironment.getRemoteEnvironment(config);
+   *     UserMainExample runnableApp = new UserMainExample();
+   *     runnableApp.run(remoteEnv, config);
+   *   }
+   *
+   */
+  @Override public StreamGraph create(Config config) {
+    StreamGraph graph = StreamGraph.fromConfig(config);
+
+    MessageStream<PageViewEvent> pageViewEvents = graph.createInStream(input1, new StringSerde("UTF-8"), new JsonSerde<>());
+    OutputStream<StatsOutput> pageViewPerMemberCounters = graph.createOutStream(output, new StringSerde("UTF-8"), new JsonSerde<StatsOutput>());
+
+    pageViewEvents.
+        partitionBy(m -> m.getMessage().memberId).
+        flatMap(new MyStatsCounter()).
+        sendTo(pageViewPerMemberCounters);
+
+    return graph;
+  }
+
+  // standalone local program model
+  public static void main(String[] args) throws Exception {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+    ExecutionEnvironment standaloneEnv = ExecutionEnvironment.getLocalEnvironment(config);
+    standaloneEnv.run(new KeyValueStoreExample(), config);
+  }
+
+  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.getMessage().timestamp) / 5) * 5;
+      String wndKey = String.format("%s-%d", message.getMessage().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.getMessage().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");
+    }
+  }
+
+  StreamSpec input1 = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "PageViewEvent");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  StreamSpec output = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "PageViewPerMember5min");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  class PageViewEvent implements MessageEnvelope<String, PageViewEvent> {
+    String pageId;
+    String memberId;
+    long timestamp;
+
+    PageViewEvent(String pageId, String memberId, long timestamp) {
+      this.pageId = pageId;
+      this.memberId = memberId;
+      this.timestamp = timestamp;
+    }
+
+    @Override
+    public String getKey() {
+      return this.pageId;
+    }
+
+    @Override
+    public PageViewEvent getMessage() {
+      return this;
+    }
+  }
+
+  class StatsOutput implements MessageEnvelope<String, 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;
+    }
+
+    @Override
+    public String getKey() {
+      return this.memberId;
+    }
+
+    @Override
+    public StatsOutput getMessage() {
+      return this;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/test/java/org/apache/samza/example/NoContextStreamExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/NoContextStreamExample.java b/samza-operator/src/test/java/org/apache/samza/example/NoContextStreamExample.java
new file mode 100644
index 0000000..577d06f
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/NoContextStreamExample.java
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.operators.OutputStream;
+import org.apache.samza.operators.StreamGraphFactory;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamSpec;
+import org.apache.samza.operators.data.InputMessageEnvelope;
+import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.Offset;
+import org.apache.samza.operators.functions.JoinFunction;
+import org.apache.samza.serializers.JsonSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.system.ExecutionEnvironment;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.util.CommandLine;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+
+/**
+ * Example {@link StreamGraphFactory} code to test the API methods
+ */
+public class NoContextStreamExample implements StreamGraphFactory {
+
+  StreamSpec input1 = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "input1");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  StreamSpec input2 = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "input2");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  StreamSpec output = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "output");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  class MessageType {
+    String joinKey;
+    List<String> joinFields = new ArrayList<>();
+  }
+
+  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
+    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
+      super(key, data, offset, partition);
+    }
+  }
+
+  private JsonMessageEnvelope getInputMessage(InputMessageEnvelope ism) {
+    return new JsonMessageEnvelope(
+        ((MessageType) ism.getMessage()).joinKey,
+        (MessageType) ism.getMessage(),
+        ism.getOffset(),
+        ism.getSystemStreamPartition());
+  }
+
+  class MyJoinFunction implements JoinFunction<String, JsonMessageEnvelope, JsonMessageEnvelope, JsonIncomingSystemMessageEnvelope<MessageType>> {
+
+    @Override
+    public JsonIncomingSystemMessageEnvelope<MessageType> apply(JsonMessageEnvelope m1,
+        JsonMessageEnvelope m2) {
+      MessageType newJoinMsg = new MessageType();
+      newJoinMsg.joinKey = m1.getKey();
+      newJoinMsg.joinFields.addAll(m1.getMessage().joinFields);
+      newJoinMsg.joinFields.addAll(m2.getMessage().joinFields);
+      return new JsonMessageEnvelope(m1.getMessage().joinKey, newJoinMsg, null, null);
+    }
+
+    @Override
+    public String getFirstKey(JsonMessageEnvelope message) {
+      return message.getKey();
+    }
+
+    @Override
+    public String getSecondKey(JsonMessageEnvelope message) {
+      return message.getKey();
+    }
+  }
+
+  /**
+   * used by remote execution environment to launch the job in remote program. The remote program should follow the similar
+   * invoking context as in standalone:
+   *
+   *   public static void main(String args[]) throws Exception {
+   *     CommandLine cmdLine = new CommandLine();
+   *     Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+   *     ExecutionEnvironment remoteEnv = ExecutionEnvironment.fromConfig(config);
+   *     remoteEnv.run(new NoContextStreamExample(), config);
+   *   }
+   *
+   */
+  @Override public StreamGraph create(Config config) {
+    StreamGraph graph = StreamGraph.fromConfig(config);
+    MessageStream<InputMessageEnvelope> inputSource1 = graph.<Object, Object, InputMessageEnvelope>createInStream(
+        input1, null, null);
+    MessageStream<InputMessageEnvelope> inputSource2 = graph.<Object, Object, InputMessageEnvelope>createInStream(
+        input2, null, null);
+    OutputStream<JsonIncomingSystemMessageEnvelope<MessageType>> outStream = graph.createOutStream(output,
+        new StringSerde("UTF-8"), new JsonSerde<>());
+
+    inputSource1.map(this::getInputMessage).
+        join(inputSource2.map(this::getInputMessage), new MyJoinFunction()).
+        sendTo(outStream);
+
+    return graph;
+  }
+
+  // standalone local program model
+  public static void main(String[] args) throws Exception {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+    ExecutionEnvironment standaloneEnv = ExecutionEnvironment.getLocalEnvironment(config);
+    standaloneEnv.run(new NoContextStreamExample(), config);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java b/samza-operator/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java
new file mode 100644
index 0000000..ad433b6
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.operators.MessageStream;
+import org.apache.samza.operators.OutputStream;
+import org.apache.samza.operators.StreamGraphFactory;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamSpec;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.functions.JoinFunction;
+import org.apache.samza.serializers.JsonSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.system.ExecutionEnvironment;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.CommandLine;
+
+import java.util.Properties;
+
+
+/**
+ * Simple 2-way stream-to-stream join example
+ */
+public class OrderShipmentJoinExample implements StreamGraphFactory {
+
+  /**
+   * used by remote execution environment to launch the job in remote program. The remote program should follow the similar
+   * invoking context as in standalone:
+   *
+   *   public static void main(String args[]) throws Exception {
+   *     CommandLine cmdLine = new CommandLine();
+   *     Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+   *     ExecutionEnvironment remoteEnv = ExecutionEnvironment.getRemoteEnvironment(config);
+   *     UserMainExample runnableApp = new UserMainExample();
+   *     runnableApp.run(remoteEnv, config);
+   *   }
+   *
+   */
+  @Override public StreamGraph create(Config config) {
+    StreamGraph graph = StreamGraph.fromConfig(config);
+
+    MessageStream<OrderRecord> orders = graph.createInStream(input1, new StringSerde("UTF-8"), new JsonSerde<>());
+    MessageStream<ShipmentRecord> shipments = graph.createInStream(input2, new StringSerde("UTF-8"), new JsonSerde<>());
+    OutputStream<FulFilledOrderRecord> fulfilledOrders = graph.createOutStream(output, new StringSerde("UTF-8"), new JsonSerde<>());
+
+    orders.join(shipments, new MyJoinFunction()).sendTo(fulfilledOrders);
+
+    return graph;
+  }
+
+  // standalone local program model
+  public static void main(String[] args) throws Exception {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+    ExecutionEnvironment standaloneEnv = ExecutionEnvironment.getLocalEnvironment(config);
+    standaloneEnv.run(new OrderShipmentJoinExample(), config);
+  }
+
+  StreamSpec input1 = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "Orders");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  StreamSpec input2 = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "Shipment");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  StreamSpec output = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "FulfilledOrders");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  class OrderRecord implements MessageEnvelope<String, OrderRecord> {
+    String orderId;
+    long orderTimeMs;
+
+    OrderRecord(String orderId, long timeMs) {
+      this.orderId = orderId;
+      this.orderTimeMs = timeMs;
+    }
+
+    @Override
+    public String getKey() {
+      return this.orderId;
+    }
+
+    @Override
+    public OrderRecord getMessage() {
+      return this;
+    }
+  }
+
+  class ShipmentRecord implements MessageEnvelope<String, ShipmentRecord> {
+    String orderId;
+    long shipTimeMs;
+
+    ShipmentRecord(String orderId, long timeMs) {
+      this.orderId = orderId;
+      this.shipTimeMs = timeMs;
+    }
+
+    @Override
+    public String getKey() {
+      return this.orderId;
+    }
+
+    @Override
+    public ShipmentRecord getMessage() {
+      return this;
+    }
+  }
+
+  class FulFilledOrderRecord implements MessageEnvelope<String, FulFilledOrderRecord> {
+    String orderId;
+    long orderTimeMs;
+    long shipTimeMs;
+
+    FulFilledOrderRecord(String orderId, long orderTimeMs, long shipTimeMs) {
+      this.orderId = orderId;
+      this.orderTimeMs = orderTimeMs;
+      this.shipTimeMs = shipTimeMs;
+    }
+
+
+    @Override
+    public String getKey() {
+      return this.orderId;
+    }
+
+    @Override
+    public FulFilledOrderRecord getMessage() {
+      return this;
+    }
+  }
+
+  FulFilledOrderRecord myJoinResult(OrderRecord m1, ShipmentRecord m2) {
+    return new FulFilledOrderRecord(m1.getKey(), m1.orderTimeMs, m2.shipTimeMs);
+  }
+
+  class MyJoinFunction implements JoinFunction<String, OrderRecord, ShipmentRecord, FulFilledOrderRecord> {
+
+    @Override
+    public FulFilledOrderRecord apply(OrderRecord message, ShipmentRecord otherMessage) {
+      return OrderShipmentJoinExample.this.myJoinResult(message, otherMessage);
+    }
+
+    @Override
+    public String getFirstKey(OrderRecord message) {
+      return message.getKey();
+    }
+
+    @Override
+    public String getSecondKey(ShipmentRecord message) {
+      return message.getKey();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/test/java/org/apache/samza/example/PageViewCounterExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/PageViewCounterExample.java b/samza-operator/src/test/java/org/apache/samza/example/PageViewCounterExample.java
new file mode 100644
index 0000000..1502aa2
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/PageViewCounterExample.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.operators.MessageStream;
+import org.apache.samza.operators.OutputStream;
+import org.apache.samza.operators.StreamGraphFactory;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamSpec;
+import org.apache.samza.operators.data.MessageEnvelope;
+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.serializers.JsonSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.system.ExecutionEnvironment;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.CommandLine;
+
+import java.time.Duration;
+import java.util.Properties;
+
+
+/**
+ * Example code to implement window-based counter
+ */
+public class PageViewCounterExample implements StreamGraphFactory {
+
+  @Override public StreamGraph create(Config config) {
+    StreamGraph graph = StreamGraph.fromConfig(config);
+
+    MessageStream<PageViewEvent> pageViewEvents = graph.createInStream(input1, new StringSerde("UTF-8"), new JsonSerde<>());
+    OutputStream<MyStreamOutput> pageViewPerMemberCounters = graph.createOutStream(output, new StringSerde("UTF-8"), new JsonSerde<>());
+
+    pageViewEvents.
+        window(Windows.<PageViewEvent, String, Integer>keyedTumblingWindow(m -> m.getMessage().memberId, Duration.ofSeconds(10), (m, c) -> c + 1).
+            setEarlyTrigger(Triggers.repeat(Triggers.count(5))).
+            setAccumulationMode(AccumulationMode.DISCARDING)).
+        map(MyStreamOutput::new).
+        sendTo(pageViewPerMemberCounters);
+    return graph;
+  }
+
+  public static void main(String[] args) {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+    ExecutionEnvironment standaloneEnv = ExecutionEnvironment.getLocalEnvironment(config);
+    standaloneEnv.run(new PageViewCounterExample(), config);
+  }
+
+  StreamSpec input1 = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "PageViewEvent");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  StreamSpec output = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "PageViewPerMember5min");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  class PageViewEvent implements MessageEnvelope<String, PageViewEvent> {
+    String pageId;
+    String memberId;
+    long timestamp;
+
+    PageViewEvent(String pageId, String memberId, long timestamp) {
+      this.pageId = pageId;
+      this.memberId = memberId;
+      this.timestamp = timestamp;
+    }
+
+    @Override
+    public String getKey() {
+      return this.pageId;
+    }
+
+    @Override
+    public PageViewEvent getMessage() {
+      return this;
+    }
+  }
+
+  class MyStreamOutput implements MessageEnvelope<String, 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();
+    }
+
+    @Override
+    public String getKey() {
+      return this.memberId;
+    }
+
+    @Override
+    public MyStreamOutput getMessage() {
+      return this;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/test/java/org/apache/samza/example/RepartitionExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/RepartitionExample.java b/samza-operator/src/test/java/org/apache/samza/example/RepartitionExample.java
new file mode 100644
index 0000000..f15e514
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/RepartitionExample.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.operators.MessageStream;
+import org.apache.samza.operators.OutputStream;
+import org.apache.samza.operators.StreamGraphFactory;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamSpec;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.windows.WindowPane;
+import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.serializers.JsonSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.system.ExecutionEnvironment;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.CommandLine;
+
+import java.time.Duration;
+import java.util.*;
+
+
+/**
+ * Example {@link StreamGraphFactory} code to test the API methods with re-partition operator
+ */
+public class RepartitionExample implements StreamGraphFactory {
+
+  /**
+   * used by remote execution environment to launch the job in remote program. The remote program should follow the similar
+   * invoking context as in standalone:
+   *
+   *   public static void main(String args[]) throws Exception {
+   *     CommandLine cmdLine = new CommandLine();
+   *     Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+   *     ExecutionEnvironment remoteEnv = ExecutionEnvironment.getRemoteEnvironment(config);
+   *     remoteEnv.run(new UserMainExample(), config);
+   *   }
+   *
+   */
+  @Override public StreamGraph create(Config config) {
+    StreamGraph graph = StreamGraph.fromConfig(config);
+
+    MessageStream<PageViewEvent> pageViewEvents = graph.createInStream(input1, new StringSerde("UTF-8"), new JsonSerde<>());
+    OutputStream<MyStreamOutput> pageViewPerMemberCounters = graph.createOutStream(output, new StringSerde("UTF-8"), new JsonSerde<>());
+
+    pageViewEvents.
+        partitionBy(m -> m.getMessage().memberId).
+        window(Windows.<PageViewEvent, String, Integer>keyedTumblingWindow(
+            msg -> msg.getMessage().memberId, Duration.ofMinutes(5), (m, c) -> c + 1)).
+        map(MyStreamOutput::new).
+        sendTo(pageViewPerMemberCounters);
+
+    return graph;
+  }
+
+  // standalone local program model
+  public static void main(String[] args) throws Exception {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+    ExecutionEnvironment standaloneEnv = ExecutionEnvironment.getLocalEnvironment(config);
+    standaloneEnv.run(new RepartitionExample(), config);
+  }
+
+  StreamSpec input1 = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "PageViewEvent");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  StreamSpec output = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "PageViewPerMember5min");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  class PageViewEvent implements MessageEnvelope<String, PageViewEvent> {
+    String pageId;
+    String memberId;
+    long timestamp;
+
+    PageViewEvent(String pageId, String memberId, long timestamp) {
+      this.pageId = pageId;
+      this.memberId = memberId;
+      this.timestamp = timestamp;
+    }
+
+    @Override
+    public String getKey() {
+      return this.pageId;
+    }
+
+    @Override
+    public PageViewEvent getMessage() {
+      return this;
+    }
+  }
+
+  class MyStreamOutput implements MessageEnvelope<String, 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();
+    }
+
+    @Override
+    public String getKey() {
+      return this.memberId;
+    }
+
+    @Override
+    public MyStreamOutput getMessage() {
+      return this;
+    }
+  }
+
+}


[07/14] samza git commit: SAMZA-1073: top-level fluent API `

Posted by ni...@apache.org.
SAMZA-1073: top-level fluent API
`


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

Branch: refs/heads/samza-fluent-api-v1
Commit: b3dd886d5928f5c2c77f6f3d582508aff401f594
Parents: 2226e3e
Author: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Authored: Thu Feb 9 01:56:10 2017 -0800
Committer: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Committed: Wed Feb 15 15:13:09 2017 -0800

----------------------------------------------------------------------
 .../apache/samza/operators/ContextManager.java  |  47 ++++
 .../apache/samza/operators/MessageStream.java   |  90 ++++---
 .../apache/samza/operators/OutputStream.java    |  41 +++
 .../org/apache/samza/operators/StreamGraph.java | 118 +++++++++
 .../samza/operators/StreamGraphFactory.java     |  38 +++
 .../samza/operators/StreamOperatorTask.java     |  51 ----
 .../org/apache/samza/operators/StreamSpec.java  |  46 ++++
 .../data/IncomingSystemMessageEnvelope.java     |  63 -----
 .../operators/data/InputMessageEnvelope.java    |  63 +++++
 .../samza/operators/data/MessageEnvelope.java   |   2 +-
 .../operators/functions/FilterFunction.java     |  23 +-
 .../operators/functions/FlatMapFunction.java    |  25 +-
 .../samza/operators/functions/InitFunction.java |  38 +++
 .../samza/operators/functions/JoinFunction.java |  48 +++-
 .../functions/KeyValueJoinFunction.java         |  44 ++++
 .../samza/operators/functions/MapFunction.java  |  25 +-
 .../samza/operators/functions/SinkFunction.java |  23 +-
 .../samza/operators/triggers/AnyTrigger.java    |   3 +-
 .../samza/operators/triggers/CountTrigger.java  |   4 +-
 .../operators/triggers/RepeatingTrigger.java    |   4 +-
 .../triggers/TimeSinceFirstMessageTrigger.java  |   3 +-
 .../triggers/TimeSinceLastMessageTrigger.java   |   4 +-
 .../samza/operators/triggers/TimeTrigger.java   |   4 +-
 .../samza/operators/triggers/Trigger.java       |   7 +-
 .../samza/operators/triggers/Triggers.java      |  41 +--
 .../apache/samza/operators/windows/Window.java  |  20 +-
 .../samza/operators/windows/WindowKey.java      |  19 +-
 .../samza/operators/windows/WindowPane.java     |   9 +-
 .../apache/samza/operators/windows/Windows.java | 136 +++++-----
 .../windows/internal/WindowInternal.java        |  14 +-
 .../samza/system/ExecutionEnvironment.java      |  73 ++++++
 .../java/org/apache/samza/task/TaskContext.java |  10 +
 .../data/TestIncomingSystemMessage.java         |   2 +-
 .../operators/windows/TestWindowOutput.java     |  35 ---
 .../samza/operators/windows/TestWindowPane.java |  33 +++
 .../samza/operators/MessageStreamImpl.java      | 151 +++++++----
 .../apache/samza/operators/StreamGraphImpl.java | 260 +++++++++++++++++++
 .../operators/StreamOperatorAdaptorTask.java    | 105 --------
 .../functions/PartialJoinFunction.java          |  65 +++++
 .../samza/operators/impl/OperatorGraph.java     | 164 ++++++++++++
 .../samza/operators/impl/OperatorImpl.java      |  22 +-
 .../samza/operators/impl/OperatorImpls.java     | 124 ---------
 .../operators/impl/PartialJoinOperatorImpl.java |  15 +-
 .../samza/operators/impl/RootOperatorImpl.java  |   7 +-
 .../impl/SessionWindowOperatorImpl.java         |  52 ++++
 .../samza/operators/impl/SinkOperatorImpl.java  |   7 +-
 .../operators/impl/StreamOperatorImpl.java      |  14 +-
 .../operators/impl/WindowOperatorImpl.java      |  11 +-
 .../samza/operators/spec/OperatorSpec.java      |  39 ++-
 .../samza/operators/spec/OperatorSpecs.java     | 161 +++++++++---
 .../operators/spec/PartialJoinOperatorSpec.java |  58 +++--
 .../samza/operators/spec/SinkOperatorSpec.java  |  70 ++++-
 .../operators/spec/StreamOperatorSpec.java      |  58 +++--
 .../operators/spec/WindowOperatorSpec.java      |  41 ++-
 .../samza/operators/spec/WindowState.java       |  16 +-
 .../system/SingleJobExecutionEnvironment.java   |  37 +++
 .../system/StandaloneExecutionEnvironment.java  |  41 +++
 .../apache/samza/task/StreamOperatorTask.java   | 108 ++++++++
 .../apache/samza/example/BroadcastGraph.java    | 121 +++++++++
 .../org/apache/samza/example/JoinGraph.java     | 118 +++++++++
 .../samza/example/KeyValueStoreExample.java     | 184 +++++++++++++
 .../samza/example/NoContextStreamExample.java   | 156 +++++++++++
 .../samza/example/OrderShipmentJoinExample.java | 190 ++++++++++++++
 .../samza/example/PageViewCounterExample.java   | 133 ++++++++++
 .../samza/example/RepartitionExample.java       | 145 +++++++++++
 .../samza/example/TestFluentStreamTasks.java    |  99 +++++++
 .../org/apache/samza/example/WindowGraph.java   |  87 +++++++
 .../apache/samza/operators/BroadcastTask.java   |  96 -------
 .../org/apache/samza/operators/JoinTask.java    |  77 ------
 .../operators/TestFluentStreamAdaptorTask.java  |  85 ------
 .../samza/operators/TestFluentStreamTasks.java  | 112 --------
 .../samza/operators/TestMessageStreamImpl.java  |  52 ++--
 .../operators/TestMessageStreamImplUtil.java    |  26 ++
 .../org/apache/samza/operators/WindowTask.java  |  63 -----
 .../samza/operators/impl/TestOperatorImpls.java |  94 +++++--
 .../operators/impl/TestSinkOperatorImpl.java    |  11 +-
 .../operators/impl/TestStreamOperatorImpl.java  |  20 +-
 .../samza/operators/spec/TestOperatorSpecs.java |  65 +++--
 78 files changed, 3455 insertions(+), 1311 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/main/java/org/apache/samza/operators/ContextManager.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/ContextManager.java b/samza-api/src/main/java/org/apache/samza/operators/ContextManager.java
new file mode 100644
index 0000000..c3b1cf3
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/ContextManager.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;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.config.Config;
+import org.apache.samza.task.TaskContext;
+
+
+/**
+ * Interface class defining methods to initialize and finalize the context used by the transformation functions.
+ */
+@InterfaceStability.Unstable
+public interface ContextManager {
+  /**
+   * The initialization method to create shared context for the whole task in Samza. Default to NO-OP
+   *
+   * @param config  the configuration object for the task
+   * @param context  the {@link TaskContext} object
+   * @return  User-defined task-wide context object
+   */
+  default TaskContext initTaskContext(Config config, TaskContext context) {
+    return context;
+  }
+
+  /**
+   * The finalize method to allow users to close resource initialized in {@link #initTaskContext} method. Default to NO-OP.
+   *
+   */
+  default void finalizeTaskContext() { }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/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 6a2f95b..87a9fd3 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
@@ -19,7 +19,6 @@
 package org.apache.samza.operators;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.operators.functions.FilterFunction;
 import org.apache.samza.operators.functions.FlatMapFunction;
 import org.apache.samza.operators.functions.JoinFunction;
@@ -29,73 +28,93 @@ import org.apache.samza.operators.windows.Window;
 import org.apache.samza.operators.windows.WindowPane;
 
 import java.util.Collection;
+import java.util.function.Function;
 
 
 /**
- * Represents a stream of {@link MessageEnvelope}s.
+ * Represents a stream of messages.
  * <p>
  * A {@link MessageStream} can be transformed into another {@link MessageStream} by applying the transforms in this API.
  *
- * @param <M>  type of {@link MessageEnvelope}s in this stream
+ * @param <M>  type of messages in this stream
  */
 @InterfaceStability.Unstable
-public interface MessageStream<M extends MessageEnvelope> {
+public interface MessageStream<M> {
 
   /**
-   * Applies the provided 1:1 {@link MapFunction} to {@link MessageEnvelope}s in this {@link MessageStream} and returns the
+   * Applies the provided 1:1 {@link Function} to messages in this {@link MessageStream} and returns the
    * transformed {@link MessageStream}.
    *
-   * @param mapFn the function to transform a {@link MessageEnvelope} to another {@link MessageEnvelope}
-   * @param <TM> the type of {@link MessageEnvelope}s in the transformed {@link MessageStream}
+   * @param mapFn the function to transform a message to another message
+   * @param <TM> the type of messages in the transformed {@link MessageStream}
    * @return the transformed {@link MessageStream}
    */
-  <TM extends MessageEnvelope> MessageStream<TM> map(MapFunction<M, TM> mapFn);
+  <TM> MessageStream<TM> map(MapFunction<M, TM> mapFn);
 
   /**
-   * Applies the provided 1:n {@link FlatMapFunction} to transform a {@link MessageEnvelope} in this {@link MessageStream}
-   * to n {@link MessageEnvelope}s in the transformed {@link MessageStream}
+   * Applies the provided 1:n {@link Function} to transform a message in this {@link MessageStream}
+   * to n messages in the transformed {@link MessageStream}
    *
-   * @param flatMapFn the function to transform a {@link MessageEnvelope} to zero or more {@link MessageEnvelope}s
-   * @param <TM> the type of {@link MessageEnvelope}s in the transformed {@link MessageStream}
+   * @param flatMapFn the function to transform a message to zero or more messages
+   * @param <TM> the type of messages in the transformed {@link MessageStream}
    * @return the transformed {@link MessageStream}
    */
-  <TM extends MessageEnvelope> MessageStream<TM> flatMap(FlatMapFunction<M, TM> flatMapFn);
+  <TM> MessageStream<TM> flatMap(FlatMapFunction<M, TM> flatMapFn);
 
   /**
-   * Applies the provided {@link FilterFunction} to {@link MessageEnvelope}s in this {@link MessageStream} and returns the
+   * Applies the provided {@link Function} to messages in this {@link MessageStream} and returns the
    * transformed {@link MessageStream}.
    * <p>
-   * The {@link FilterFunction} is a predicate which determines whether a {@link MessageEnvelope} in this {@link MessageStream}
+   * The {@link Function} is a predicate which determines whether a message in this {@link MessageStream}
    * should be retained in the transformed {@link MessageStream}.
    *
-   * @param filterFn the predicate to filter {@link MessageEnvelope}s from this {@link MessageStream}
+   * @param filterFn the predicate to filter messages from this {@link MessageStream}
    * @return the transformed {@link MessageStream}
    */
   MessageStream<M> filter(FilterFunction<M> filterFn);
 
   /**
-   * Allows sending {@link MessageEnvelope}s in this {@link MessageStream} to an output
-   * {@link org.apache.samza.system.SystemStream} using the provided {@link SinkFunction}.
+   * Allows sending messages in this {@link MessageStream} to an output using the provided {@link SinkFunction}.
    *
-   * @param sinkFn the function to send {@link MessageEnvelope}s in this stream to output systems
+   * NOTE: the output may not be a {@link org.apache.samza.system.SystemStream}. It can be an external database, etc.
+   *
+   * @param sinkFn  the function to send messages in this stream to output
    */
   void sink(SinkFunction<M> sinkFn);
 
   /**
-   * Groups and processes the {@link MessageEnvelope}s in this {@link MessageStream} according to the provided {@link Window}
+   * Allows sending messages in this {@link MessageStream} to an output {@link MessageStream}.
+   *
+   * NOTE: the {@code stream} has to be a {@link MessageStream}.
+   *
+   * @param stream  the output {@link MessageStream}
+   */
+  void sendTo(OutputStream<M> stream);
+
+  /**
+   * Allows sending messages to an intermediate {@link MessageStream}.
+   *
+   * NOTE: the {@code stream} has to be a {@link MessageStream}.
+   *
+   * @param stream  the intermediate {@link MessageStream} to send the message to
+   * @return  the intermediate {@link MessageStream} to consume the messages sent
+   */
+  MessageStream<M> sendThrough(OutputStream<M> stream);
+
+  /**
+   * Groups the messages in this {@link MessageStream} according to the provided {@link Window} semantics
    * (e.g. tumbling, sliding or session windows) and returns the transformed {@link MessageStream} of
    * {@link WindowPane}s.
    * <p>
    * Use the {@link org.apache.samza.operators.windows.Windows} helper methods to create the appropriate windows.
    *
-   * @param window the window to group and process {@link MessageEnvelope}s from this {@link MessageStream}
-   * @param <K> the type of key in the {@link MessageEnvelope} in this {@link MessageStream}. If a key is specified,
+   * @param window the window to group and process messages from this {@link MessageStream}
+   * @param <K> the type of key in the message in this {@link MessageStream}. If a key is specified,
    *            panes are emitted per-key.
    * @param <WV> the type of value in the {@link WindowPane} in the transformed {@link MessageStream}
-   * @param <WM> the type of {@link WindowPane} in the transformed {@link MessageStream}
    * @return the transformed {@link MessageStream}
    */
-  <K, WV, WM extends WindowPane<K, WV>> MessageStream<WM> window(Window<M, K, WV, WM> window);
+  <K, WV> MessageStream<WindowPane<K, WV>> window(Window<M, K, WV> window);
 
   /**
    * Joins this {@link MessageStream} with another {@link MessageStream} using the provided pairwise {@link JoinFunction}.
@@ -103,23 +122,32 @@ public interface MessageStream<M extends MessageEnvelope> {
    * We currently only support 2-way joins.
    *
    * @param otherStream the other {@link MessageStream} to be joined with
-   * @param joinFn the function to join {@link MessageEnvelope}s from this and the other {@link MessageStream}
+   * @param joinFn the function to join messages from this and the other {@link MessageStream}
    * @param <K> the type of join key
-   * @param <OM> the type of {@link MessageEnvelope}s in the other stream
-   * @param <RM> the type of {@link MessageEnvelope}s resulting from the {@code joinFn}
+   * @param <OM> the type of messages in the other stream
+   * @param <RM> the type of messages resulting from the {@code joinFn}
    * @return the joined {@link MessageStream}
    */
-  <K, OM extends MessageEnvelope<K, ?>, RM extends MessageEnvelope> MessageStream<RM> join(MessageStream<OM> otherStream,
-      JoinFunction<M, OM, RM> joinFn);
+  <K, OM, RM> MessageStream<RM> join(MessageStream<OM> otherStream, JoinFunction<K, M, OM, RM> joinFn);
 
   /**
    * Merge all {@code otherStreams} with this {@link MessageStream}.
    * <p>
-   * The merging streams must have the same {@link MessageEnvelope} type {@code M}.
+   * The merging streams must have the same messages of type {@code M}.
    *
    * @param otherStreams  other {@link MessageStream}s to be merged with this {@link MessageStream}
    * @return  the merged {@link MessageStream}
    */
   MessageStream<M> merge(Collection<MessageStream<M>> otherStreams);
-  
+
+  /**
+   * Send the input message to an output {@link org.apache.samza.system.SystemStream} and consume it as input {@link MessageStream} again.
+   *
+   * Note: this is an transform function only used in logic DAG. In a physical DAG, this is either translated to a NOOP function, or a {@code MessageStream#sendThrough} function.
+   *
+   * @param parKeyExtractor  a {@link Function} that extract the partition key from a message in this {@link MessageStream}
+   * @param <K>  the type of partition key
+   * @return  a {@link MessageStream} object after the re-partition
+   */
+  <K> MessageStream<M> partitionBy(Function<M, K> parKeyExtractor);
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/main/java/org/apache/samza/operators/OutputStream.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/OutputStream.java b/samza-api/src/main/java/org/apache/samza/operators/OutputStream.java
new file mode 100644
index 0000000..179f0e7
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/OutputStream.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.operators.functions.SinkFunction;
+
+
+/**
+ * The interface class defining the specific {@link SinkFunction} for a system {@link OutputStream}.
+ *
+ * @param <M>  The type of message to be send to this output stream
+ */
+@InterfaceStability.Unstable
+public interface OutputStream<M> {
+
+  /**
+   * Returns the specific {@link SinkFunction} for this output stream. The {@link OutputStream} is created
+   * via {@link StreamGraph#createOutStream(StreamSpec, Serde, Serde)} or {@link StreamGraph#createIntStream(StreamSpec, Serde, Serde)}.
+   * Hence, the proper types of serdes for key and value are instantiated and are used in the {@link SinkFunction} returned.
+   *
+   * @return  The pre-defined {@link SinkFunction} to apply proper serdes before sending the message to the output stream.
+   */
+  SinkFunction<M> getSinkFunction();
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java b/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java
new file mode 100644
index 0000000..9e6644b
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.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.operators;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.ConfigException;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.serializers.Serde;
+
+import java.util.Map;
+
+
+/**
+ * Job-level programming interface to create an operator DAG and run in various different runtime environments.
+ */
+@InterfaceStability.Unstable
+public interface StreamGraph {
+  /**
+   * Method to add an input {@link MessageStream} from the system
+   *
+   * @param streamSpec  the {@link StreamSpec} describing the physical characteristics of the input {@link MessageStream}
+   * @param keySerde  the serde used to serialize/deserialize the message key from the input {@link MessageStream}
+   * @param msgSerde  the serde used to serialize/deserialize the message body from the input {@link MessageStream}
+   * @param <K>  the type of key in the input message
+   * @param <V>  the type of message in the input message
+   * @param <M>  the type of {@link MessageEnvelope} in the input {@link MessageStream}
+   * @return   the input {@link MessageStream} object
+   */
+  <K, V, M extends MessageEnvelope<K, V>> MessageStream<M> createInStream(StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde);
+
+  /**
+   * Method to add an output {@link MessageStream} from the system
+   *
+   * @param streamSpec  the {@link StreamSpec} describing the physical characteristics of the output {@link MessageStream}
+   * @param keySerde  the serde used to serialize/deserialize the message key from the output {@link MessageStream}
+   * @param msgSerde  the serde used to serialize/deserialize the message body from the output {@link MessageStream}
+   * @param <K>  the type of key in the output message
+   * @param <V>  the type of message in the output message
+   * @param <M>  the type of {@link MessageEnvelope} in the output {@link MessageStream}
+   * @return   the output {@link MessageStream} object
+   */
+  <K, V, M extends MessageEnvelope<K, V>> OutputStream<M> createOutStream(StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde);
+
+  /**
+   * Method to add an intermediate {@link MessageStream} from the system
+   *
+   * @param streamSpec  the {@link StreamSpec} describing the physical characteristics of the intermediate {@link MessageStream}
+   * @param keySerde  the serde used to serialize/deserialize the message key from the intermediate {@link MessageStream}
+   * @param msgSerde  the serde used to serialize/deserialize the message body from the intermediate {@link MessageStream}
+   * @param <K>  the type of key in the intermediate message
+   * @param <V>  the type of message in the intermediate message
+   * @param <M>  the type of {@link MessageEnvelope} in the intermediate {@link MessageStream}
+   * @return   the intermediate {@link MessageStream} object
+   */
+  <K, V, M extends MessageEnvelope<K, V>> OutputStream<M> createIntStream(StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde);
+
+  /**
+   * Method to get the input {@link MessageStream}s
+   *
+   * @return the input {@link MessageStream}
+   */
+  Map<StreamSpec, MessageStream> getInStreams();
+
+  /**
+   * Method to get the {@link OutputStream}s
+   *
+   * @return  the map of all {@link OutputStream}s
+   */
+  Map<StreamSpec, OutputStream> getOutStreams();
+
+  /**
+   * Method to set the {@link ContextManager} for this {@link StreamGraph}
+   *
+   * @param manager  the {@link ContextManager} object
+   * @return  this {@link StreamGraph} object
+   */
+  StreamGraph withContextManager(ContextManager manager);
+
+  String GRAPH_CONFIG = "job.stream.graph.impl.class";
+  String DEFAULT_GRAPH_IMPL_CLASS = "org.apache.samza.operators.StreamGraphImpl";
+
+  /**
+   * Static method to instantiate the implementation class of {@link StreamGraph}.
+   *
+   * @param config  the {@link Config} object for this job
+   * @return  the {@link StreamGraph} object created
+   */
+  static StreamGraph fromConfig(Config config) {
+
+    try {
+      if (StreamGraph.class.isAssignableFrom(Class.forName(config.get(GRAPH_CONFIG, DEFAULT_GRAPH_IMPL_CLASS)))) {
+        return (StreamGraph) Class.forName(config.get(GRAPH_CONFIG, DEFAULT_GRAPH_IMPL_CLASS)).newInstance();
+      }
+    } catch (Exception e) {
+      throw new ConfigException(String.format("Problem in loading StreamGraphImpl class %s", config.get(GRAPH_CONFIG)), e);
+    }
+    throw new ConfigException(String.format(
+        "Class %s does not implement interface StreamGraphBuilder properly",
+        config.get(GRAPH_CONFIG)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/main/java/org/apache/samza/operators/StreamGraphFactory.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/StreamGraphFactory.java b/samza-api/src/main/java/org/apache/samza/operators/StreamGraphFactory.java
new file mode 100644
index 0000000..c292363
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/StreamGraphFactory.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;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.config.Config;
+
+
+/**
+ * This interface defines a factory class that user will implement to create user-defined operator DAG in a {@link StreamGraph} object.
+ */
+@InterfaceStability.Unstable
+public interface StreamGraphFactory {
+  /**
+   * Users are required to implement this abstract method to initialize the processing logic of the application, in terms
+   * of a DAG of {@link org.apache.samza.operators.MessageStream}s and operators
+   *
+   * @param config  the {@link Config} of the application
+   * @return  the {@link StreamGraph} object which contains user-defined processing logic of the application
+   */
+  StreamGraph create(Config config);
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/main/java/org/apache/samza/operators/StreamOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/StreamOperatorTask.java b/samza-api/src/main/java/org/apache/samza/operators/StreamOperatorTask.java
deleted file mode 100644
index 16cf27a..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/StreamOperatorTask.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators;
-
-import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.IncomingSystemMessageEnvelope;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.util.Map;
-
-
-/**
- * A {@link StreamOperatorTask} is the basic interface to implement for processing {@link MessageStream}s.
- * Implementations can describe the transformation steps for each {@link MessageStream} in the
- * {@link #transform} method using {@link MessageStream} APIs.
- * <p>
- * Implementations may be augmented by implementing {@link org.apache.samza.task.InitableTask},
- * {@link org.apache.samza.task.WindowableTask} and {@link org.apache.samza.task.ClosableTask} interfaces,
- * but should not implement {@link org.apache.samza.task.StreamTask} or {@link org.apache.samza.task.AsyncStreamTask}
- * interfaces.
- */
-@InterfaceStability.Unstable
-public interface StreamOperatorTask {
-
-  /**
-   * Describe the transformation steps for each {@link MessageStream}s for this task using the
-   * {@link MessageStream} APIs. Each {@link MessageStream} corresponds to one {@link SystemStreamPartition}
-   * in the input system.
-   *
-   * @param messageStreams the {@link MessageStream}s that receive {@link IncomingSystemMessageEnvelope}s
-   *                       from their corresponding {@link org.apache.samza.system.SystemStreamPartition}
-   */
-  void transform(Map<SystemStreamPartition, MessageStream<IncomingSystemMessageEnvelope>> messageStreams);
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/main/java/org/apache/samza/operators/StreamSpec.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/StreamSpec.java b/samza-api/src/main/java/org/apache/samza/operators/StreamSpec.java
new file mode 100644
index 0000000..c8a5e8d
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/StreamSpec.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.system.SystemStream;
+
+import java.util.Properties;
+
+
+/**
+ * This interface defines the specification of a {@link SystemStream}. It will be used by the {@link org.apache.samza.system.SystemAdmin}
+ * to create a {@link SystemStream}
+ */
+@InterfaceStability.Unstable
+public interface StreamSpec {
+  /**
+   * Get the {@link SystemStream}
+   *
+   * @return  {@link SystemStream} object
+   */
+  SystemStream getSystemStream();
+
+  /**
+   * Get the physical properties of the {@link SystemStream}
+   *
+   * @return  the properties of this stream
+   */
+  Properties getProperties();
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessageEnvelope.java b/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessageEnvelope.java
deleted file mode 100644
index a65809c..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessageEnvelope.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators.data;
-
-import org.apache.samza.system.IncomingMessageEnvelope;
-import org.apache.samza.system.SystemStreamPartition;
-
-
-/**
- * A {@link MessageEnvelope} that provides additional information about its input {@link SystemStreamPartition}
- * and its {@link Offset} within the {@link SystemStreamPartition}.
- * <p>
- * Note: the {@link Offset} is only unique and comparable within its {@link SystemStreamPartition}.
- */
-public class IncomingSystemMessageEnvelope implements MessageEnvelope<Object, Object> {
-
-  private final IncomingMessageEnvelope ime;
-
-  /**
-   * Creates an {@code IncomingSystemMessageEnvelope} from the {@link IncomingMessageEnvelope}.
-   *
-   * @param ime  the {@link IncomingMessageEnvelope} from the input system.
-   */
-  public IncomingSystemMessageEnvelope(IncomingMessageEnvelope ime) {
-    this.ime = ime;
-  }
-
-  @Override
-  public Object getKey() {
-    return this.ime.getKey();
-  }
-
-  @Override
-  public Object getMessage() {
-    return this.ime.getMessage();
-  }
-
-  public Offset getOffset() {
-    // TODO: need to add offset factory to generate different types of offset. This is just a placeholder,
-    // assuming incoming message envelope carries long value as offset (i.e. Kafka case)
-    return new LongOffset(this.ime.getOffset());
-  }
-
-  public SystemStreamPartition getSystemStreamPartition() {
-    return this.ime.getSystemStreamPartition();
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/main/java/org/apache/samza/operators/data/InputMessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/data/InputMessageEnvelope.java b/samza-api/src/main/java/org/apache/samza/operators/data/InputMessageEnvelope.java
new file mode 100644
index 0000000..306145b
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/data/InputMessageEnvelope.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.data;
+
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.SystemStreamPartition;
+
+
+/**
+ * A {@link MessageEnvelope} that provides additional information about its input {@link SystemStreamPartition}
+ * and its {@link Offset} within the {@link SystemStreamPartition}.
+ * <p>
+ * Note: the {@link Offset} is only unique and comparable within its {@link SystemStreamPartition}.
+ */
+public class InputMessageEnvelope implements MessageEnvelope<Object, Object> {
+
+  private final IncomingMessageEnvelope ime;
+
+  /**
+   * Creates an {@code InputMessageEnvelope} from the {@link IncomingMessageEnvelope}.
+   *
+   * @param ime  the {@link IncomingMessageEnvelope} from the input system.
+   */
+  public InputMessageEnvelope(IncomingMessageEnvelope ime) {
+    this.ime = ime;
+  }
+
+  @Override
+  public Object getKey() {
+    return this.ime.getKey();
+  }
+
+  @Override
+  public Object getMessage() {
+    return this.ime.getMessage();
+  }
+
+  public Offset getOffset() {
+    // TODO: need to add offset factory to generate different types of offset. This is just a placeholder,
+    // assuming incoming message envelope carries long value as offset (i.e. Kafka case)
+    return new LongOffset(this.ime.getOffset());
+  }
+
+  public SystemStreamPartition getSystemStreamPartition() {
+    return this.ime.getSystemStreamPartition();
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/main/java/org/apache/samza/operators/data/MessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/data/MessageEnvelope.java b/samza-api/src/main/java/org/apache/samza/operators/data/MessageEnvelope.java
index ad64231..703a44c 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/data/MessageEnvelope.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/data/MessageEnvelope.java
@@ -23,7 +23,7 @@ import org.apache.samza.annotation.InterfaceStability;
 
 
 /**
- * An entry in the input/output {@link org.apache.samza.operators.MessageStream}s.
+ * An entry in the input/output {@link org.apache.samza.operators.MessageStream}s
  */
 @InterfaceStability.Unstable
 public interface MessageEnvelope<K, M> {

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/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 e611cd0..73c5c9d 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
@@ -19,22 +19,29 @@
 package org.apache.samza.operators.functions;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.config.Config;
+import org.apache.samza.task.TaskContext;
 
 
 /**
- * A function that specifies whether a {@link MessageEnvelope} should be retained for further processing or filtered out.
- * @param <M>  type of the input {@link MessageEnvelope}
+ * A function that specifies whether a message should be retained for further processing or filtered out.
+ * @param <M>  type of the input message
  */
 @InterfaceStability.Unstable
-@FunctionalInterface
-public interface FilterFunction<M extends MessageEnvelope> {
+public interface FilterFunction<M> extends InitFunction {
 
   /**
-   * Returns a boolean indicating whether this {@link MessageEnvelope} should be retained or filtered out.
-   * @param message  the {@link MessageEnvelope} to be checked
-   * @return  true if {@link MessageEnvelope} should be retained
+   * Returns a boolean indicating whether this message should be retained or filtered out.
+   * @param message  the input message to be checked
+   * @return  true if {@code message} should be retained
    */
   boolean apply(M message);
 
+  /**
+   * Init method to initialize the context for this {@link FilterFunction}. The default implementation is NO-OP.
+   *
+   * @param config  the {@link Config} object for this task
+   * @param context  the {@link TaskContext} object for this task
+   */
+  default void init(Config config, TaskContext context) { }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/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 dbc0bd9..f8458f2 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
@@ -19,26 +19,33 @@
 package org.apache.samza.operators.functions;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.config.Config;
+import org.apache.samza.task.TaskContext;
 
 import java.util.Collection;
 
 
 /**
- * A function that transforms a {@link MessageEnvelope} into a collection of 0 or more {@link MessageEnvelope}s,
+ * A function that transforms an input message into a collection of 0 or more messages,
  * possibly of a different type.
- * @param <M>  type of the input {@link MessageEnvelope}
- * @param <OM>  type of the transformed {@link MessageEnvelope}s
+ * @param <M>  type of the input message
+ * @param <OM>  type of the transformed messages
  */
 @InterfaceStability.Unstable
-@FunctionalInterface
-public interface FlatMapFunction<M extends MessageEnvelope, OM extends MessageEnvelope> {
+public interface FlatMapFunction<M, OM>  extends InitFunction {
 
   /**
-   * Transforms the provided {@link MessageEnvelope} into a collection of 0 or more {@link MessageEnvelope}s.
-   * @param message  the {@link MessageEnvelope} to be transformed
-   * @return  a collection of 0 or more transformed {@link MessageEnvelope}s
+   * Transforms the provided message into a collection of 0 or more messages.
+   * @param message  the input message to be transformed
+   * @return  a collection of 0 or more transformed messages
    */
   Collection<OM> apply(M message);
 
+  /**
+   * Init method to initialize the context for this {@link FlatMapFunction}. The default implementation is NO-OP.
+   *
+   * @param config  the {@link Config} object for this task
+   * @param context  the {@link TaskContext} object for this task
+   */
+  default void init(Config config, TaskContext context) { };
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/main/java/org/apache/samza/operators/functions/InitFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/InitFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/InitFunction.java
new file mode 100644
index 0000000..eec56df
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/InitFunction.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 org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.config.Config;
+import org.apache.samza.task.TaskContext;
+
+
+/**
+ * interface defined to initalize the context of message transformation functions
+ */
+@InterfaceStability.Unstable
+public interface InitFunction {
+  /**
+   * Interface method to initialize the context for a specific message transformation function.
+   *
+   * @param config  the {@link Config} object for this task
+   * @param context  the {@link TaskContext} object for this task
+   */
+  void init(Config config, TaskContext context);
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/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 8cb1fce..afc92ee 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
@@ -19,26 +19,50 @@
 package org.apache.samza.operators.functions;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.config.Config;
+import org.apache.samza.task.TaskContext;
 
 
 /**
- * A function that joins {@link MessageEnvelope}s from two {@link org.apache.samza.operators.MessageStream}s and produces
- * a joined {@link MessageEnvelope}.
- * @param <M>  type of the input {@link MessageEnvelope}
- * @param <JM>  type of the {@link MessageEnvelope} to join with
- * @param <RM>  type of the joined {@link MessageEnvelope}
+ * A function that joins messages from two {@link org.apache.samza.operators.MessageStream}s and produces
+ * a joined message.
+ * @param <K>  type of the join key
+ * @param <M>  type of the input message
+ * @param <JM>  type of the message to join with
+ * @param <RM>  type of the joined message
  */
 @InterfaceStability.Unstable
-@FunctionalInterface
-public interface JoinFunction<M extends MessageEnvelope, JM extends MessageEnvelope, RM extends MessageEnvelope> {
+public interface JoinFunction<K, M, JM, RM>  extends InitFunction {
 
   /**
-   * Join the provided {@link MessageEnvelope}s and produces the joined {@link MessageEnvelope}.
-   * @param message  the input {@link MessageEnvelope}
-   * @param otherMessage  the {@link MessageEnvelope} to join with
-   * @return  the joined {@link MessageEnvelope}
+   * Join the provided input messages and produces the joined messages.
+   * @param message  the input message
+   * @param otherMessage  the message to join with
+   * @return  the joined message
    */
   RM apply(M message, JM otherMessage);
 
+  /**
+   * Method to get the join key in the messages from the first input stream
+   *
+   * @param message  the input message from the first input stream
+   * @return  the join key
+   */
+  K getFirstKey(M message);
+
+  /**
+   * Method to get the join key in the messages from the second input stream
+   *
+   * @param message  the input message from the second input stream
+   * @return  the join key
+   */
+  K getSecondKey(JM message);
+
+  /**
+   * Init method to initialize the context for this {@link JoinFunction}. The default implementation is NO-OP.
+   *
+   * @param config  the {@link Config} object for this task
+   * @param context  the {@link TaskContext} object for this task
+   */
+  default void init(Config config, TaskContext context) { }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/main/java/org/apache/samza/operators/functions/KeyValueJoinFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/KeyValueJoinFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/KeyValueJoinFunction.java
new file mode 100644
index 0000000..b651b3d
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/KeyValueJoinFunction.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.functions;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.operators.data.MessageEnvelope;
+
+/**
+ * A specific {@link JoinFunction} that joins {@link MessageEnvelope}s from two {@link org.apache.samza.operators.MessageStream}s and produces
+ * a joined message.
+ *
+ * @param <K>  type of the join key
+ * @param <M>  type of the input {@link MessageEnvelope}
+ * @param <JM>  type of the {@link MessageEnvelope} to join with
+ * @param <RM>  type of the joined message
+ */
+@InterfaceStability.Unstable
+@FunctionalInterface
+public interface KeyValueJoinFunction<K, M extends MessageEnvelope<K, ?>, JM extends MessageEnvelope<K, ?>, RM> extends JoinFunction<K, M, JM, RM> {
+
+  default K getFirstKey(M message) {
+    return message.getKey();
+  }
+
+  default K getSecondKey(JM message) {
+    return message.getKey();
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/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 04919a7..a051914 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
@@ -19,23 +19,30 @@
 package org.apache.samza.operators.functions;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.config.Config;
+import org.apache.samza.task.TaskContext;
 
 
 /**
- * A function that transforms a {@link MessageEnvelope} into another {@link MessageEnvelope}, possibly of a different type.
- * @param <M>  type of the input {@link MessageEnvelope}
- * @param <OM>  type of the transformed {@link MessageEnvelope}
+ * A function that transforms an input message into another message, possibly of a different type.
+ * @param <M>  type of the input message
+ * @param <OM>  type of the transformed message
  */
 @InterfaceStability.Unstable
-@FunctionalInterface
-public interface MapFunction<M extends MessageEnvelope, OM extends MessageEnvelope> {
+public interface MapFunction<M, OM>  extends InitFunction {
 
   /**
-   * Transforms the provided {@link MessageEnvelope} into another {@link MessageEnvelope}
-   * @param message  the {@link MessageEnvelope} to be transformed
-   * @return  the transformed {@link MessageEnvelope}
+   * Transforms the provided message into another message
+   * @param message  the input message to be transformed
+   * @return  the transformed message
    */
   OM apply(M message);
 
+  /**
+   * Init method to initialize the context for this {@link MapFunction}. The default implementation is NO-OP.
+   *
+   * @param config  the {@link Config} object for this task
+   * @param context  the {@link TaskContext} object for this task
+   */
+  default void init(Config config, TaskContext context) { }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/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 505da92..1050771 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
@@ -19,28 +19,35 @@
 package org.apache.samza.operators.functions;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.config.Config;
 import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
 
 
 /**
- * A function that allows sending a {@link MessageEnvelope} to an output system.
- * @param <M>  type of the input {@link MessageEnvelope}
+ * A function that allows sending a message to an output system.
+ * @param <M>  type of the input message
  */
 @InterfaceStability.Unstable
-@FunctionalInterface
-public interface SinkFunction<M extends MessageEnvelope> {
+public interface SinkFunction<M>  extends InitFunction {
 
   /**
-   * Allows sending the provided {@link MessageEnvelope} to an output {@link org.apache.samza.system.SystemStream} using
+   * Allows sending the provided message to an output {@link org.apache.samza.system.SystemStream} using
    * the provided {@link MessageCollector}. Also provides access to the {@link TaskCoordinator} to request commits
    * or shut the container down.
    *
-   * @param message  the {@link MessageEnvelope} to be sent to an output {@link org.apache.samza.system.SystemStream}
-   * @param messageCollector  the {@link MessageCollector} to use to send the {@link MessageEnvelope}
+   * @param message  the input message to be sent to an output {@link org.apache.samza.system.SystemStream}
+   * @param messageCollector  the {@link MessageCollector} to use to send the {@link org.apache.samza.operators.data.MessageEnvelope}
    * @param taskCoordinator  the {@link TaskCoordinator} to request commits or shutdown
    */
   void apply(M message, MessageCollector messageCollector, TaskCoordinator taskCoordinator);
 
+  /**
+   * Init method to initialize the context for this {@link MapFunction}. The default implementation is NO-OP.
+   *
+   * @param config  the {@link Config} object for this task
+   * @param context  the {@link TaskContext} object for this task
+   */
+  default void init(Config config, TaskContext context) { }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/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 3ca4e9a..6e134df 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,13 +18,12 @@
 */
 package org.apache.samza.operators.triggers;
 
-import org.apache.samza.operators.data.MessageEnvelope;
 import java.util.List;
 
 /**
  * A {@link Trigger} fires as soon as any of its individual triggers has fired.
  */
-public class AnyTrigger<M extends MessageEnvelope> implements Trigger {
+public class AnyTrigger<M> implements Trigger {
 
   private final List<Trigger> triggers;
 

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/main/java/org/apache/samza/operators/triggers/CountTrigger.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/triggers/CountTrigger.java b/samza-api/src/main/java/org/apache/samza/operators/triggers/CountTrigger.java
index ba14928..1cf930c 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/triggers/CountTrigger.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/triggers/CountTrigger.java
@@ -18,13 +18,11 @@
  */
 package org.apache.samza.operators.triggers;
 
-import org.apache.samza.operators.data.MessageEnvelope;
-
 /**
  * A {@link Trigger} that fires when the number of messages in the {@link org.apache.samza.operators.windows.WindowPane}
  * reaches the specified count.
  */
-public class CountTrigger<M extends MessageEnvelope> implements Trigger {
+public class CountTrigger<M> implements Trigger {
 
   private final long count;
 

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/main/java/org/apache/samza/operators/triggers/RepeatingTrigger.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/triggers/RepeatingTrigger.java b/samza-api/src/main/java/org/apache/samza/operators/triggers/RepeatingTrigger.java
index ae9564d..7f78eb8 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/triggers/RepeatingTrigger.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/triggers/RepeatingTrigger.java
@@ -18,12 +18,10 @@
  */
 package org.apache.samza.operators.triggers;
 
-import org.apache.samza.operators.data.MessageEnvelope;
-
 /**
  * A {@link Trigger} that repeats its underlying trigger forever.
  */
-class RepeatingTrigger<M extends MessageEnvelope> implements Trigger<M> {
+class RepeatingTrigger<M> implements Trigger<M> {
 
   private final Trigger<M> trigger;
 

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeSinceFirstMessageTrigger.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeSinceFirstMessageTrigger.java b/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeSinceFirstMessageTrigger.java
index 13fc3cd..4de60a2 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeSinceFirstMessageTrigger.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeSinceFirstMessageTrigger.java
@@ -19,7 +19,6 @@
 
 package org.apache.samza.operators.triggers;
 
-import org.apache.samza.operators.data.MessageEnvelope;
 
 import java.time.Duration;
 
@@ -27,7 +26,7 @@ import java.time.Duration;
  * A {@link Trigger} that fires after the specified duration has passed since the first {@link MessageEnvelope} in
  * the window pane.
  */
-public class TimeSinceFirstMessageTrigger<M extends MessageEnvelope> implements Trigger {
+public class TimeSinceFirstMessageTrigger<M> implements Trigger {
 
   private final Duration duration;
   private final DurationCharacteristic characteristic = DurationCharacteristic.PROCESSING_TIME;

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeSinceLastMessageTrigger.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeSinceLastMessageTrigger.java b/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeSinceLastMessageTrigger.java
index 0150d86..6b09625 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeSinceLastMessageTrigger.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeSinceLastMessageTrigger.java
@@ -18,14 +18,12 @@
  */
 package org.apache.samza.operators.triggers;
 
-import org.apache.samza.operators.data.MessageEnvelope;
-
 import java.time.Duration;
 
 /*
  * A {@link Trigger} that fires when there are no new {@link MessageEnvelope}s in the window pane for the specified duration.
  */
-public class TimeSinceLastMessageTrigger<M extends MessageEnvelope> implements Trigger {
+public class TimeSinceLastMessageTrigger<M> implements Trigger {
 
   private final Duration duration;
   private final DurationCharacteristic characteristic = DurationCharacteristic.PROCESSING_TIME;

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeTrigger.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeTrigger.java b/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeTrigger.java
index ed7fef7..c5875aa 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeTrigger.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeTrigger.java
@@ -18,14 +18,12 @@
  */
 package org.apache.samza.operators.triggers;
 
-import org.apache.samza.operators.data.MessageEnvelope;
-
 import java.time.Duration;
 
 /*
  * A {@link Trigger} that fires after the specified duration in processing time.
  */
-public class TimeTrigger<M extends MessageEnvelope> implements Trigger {
+public class TimeTrigger<M> implements Trigger {
 
   private final Duration duration;
   private final DurationCharacteristic characteristic = DurationCharacteristic.PROCESSING_TIME;

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/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 6dc4f43..be0a877 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,15 +20,16 @@
 package org.apache.samza.operators.triggers;
 
 
-import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.annotation.InterfaceStability;
 
 /**
  * Marker interface for all triggers. The firing of a trigger indicates the completion of a window pane.
  *
  * <p> Use the {@link Triggers} APIs to create a {@link Trigger}.
  *
- * @param <M> the type of the incoming {@link MessageEnvelope}
+ * @param <M> the type of the incoming message
  */
-public interface Trigger<M extends MessageEnvelope> {
+@InterfaceStability.Unstable
+public interface Trigger<M> {
 
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/main/java/org/apache/samza/operators/triggers/Triggers.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/triggers/Triggers.java b/samza-api/src/main/java/org/apache/samza/operators/triggers/Triggers.java
index f27cfd8..97fb7b7 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/triggers/Triggers.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/triggers/Triggers.java
@@ -19,7 +19,6 @@
 package org.apache.samza.operators.triggers;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.MessageEnvelope;
 
 import java.time.Duration;
 import java.util.ArrayList;
@@ -35,61 +34,63 @@ import java.util.List;
  * <pre> {@code
  *   MessageStream<> windowedStream = stream.window(Windows.tumblingWindow(Duration.of(10, TimeUnit.SECONDS))
  *     .setEarlyTrigger(Triggers.repeat(Triggers.any(Triggers.count(50), Triggers.timeSinceFirstMessage(Duration.of(4, TimeUnit.SECONDS))))))
- *     .accumulateFiredPanes());
+ *     .setAccumulationMode(AccumulationMode.ACCUMULATING));
  * }</pre>
  *
- * @param <M> the type of input {@link MessageEnvelope}s in the {@link org.apache.samza.operators.MessageStream}
  */
 @InterfaceStability.Unstable
-public final class Triggers<M extends MessageEnvelope> {
+public final class Triggers {
 
   private Triggers() { }
 
   /**
-   * Creates a {@link Trigger} that fires when the number of {@link MessageEnvelope}s in the pane
+   * Creates a {@link Trigger} that fires when the number of messages in the pane
    * reaches the specified count.
    *
-   * @param count the number of {@link MessageEnvelope}s to fire the trigger after
+   * @param count the number of messages to fire the trigger after
+   * @param <M> the type of input message in the window
    * @return the created trigger
    */
-  public static Trigger count(long count) {
-    return new CountTrigger(count);
+  public static <M> Trigger<M> count(long count) {
+    return new CountTrigger<M>(count);
   }
 
   /**
-   * Creates a trigger that fires after the specified duration has passed since the first {@link MessageEnvelope} in
+   * Creates a trigger that fires after the specified duration has passed since the first message in
    * the pane.
    *
    * @param duration the duration since the first element
+   * @param <M> the type of input message in the window
    * @return the created trigger
    */
-  public static Trigger timeSinceFirstMessage(Duration duration) {
-    return new TimeSinceFirstMessageTrigger(duration);
+  public static <M> Trigger<M> timeSinceFirstMessage(Duration duration) {
+    return new TimeSinceFirstMessageTrigger<M>(duration);
   }
 
   /**
-   * Creates a trigger that fires when there is no new {@link MessageEnvelope} for the specified duration in the pane.
+   * Creates a trigger that fires when there is no new message for the specified duration in the pane.
    *
    * @param duration the duration since the last element
+   * @param <M> the type of input message in the window
    * @return the created trigger
    */
-  public static Trigger timeSinceLastMessage(Duration duration) {
-    return new TimeSinceLastMessageTrigger(duration);
+  public static <M> Trigger<M> timeSinceLastMessage(Duration duration) {
+    return new TimeSinceLastMessageTrigger<M>(duration);
   }
 
   /**
    * Creates a trigger that fires when any of the provided triggers fire.
    *
-   * @param <M> the type of input {@link MessageEnvelope} in the window
    * @param triggers the individual triggers
+   * @param <M> the type of input message in the window
    * @return the created trigger
    */
-  public static <M extends MessageEnvelope> Trigger any(Trigger<M>... triggers) {
-    List<Trigger> triggerList = new ArrayList<>();
+  public static <M> Trigger<M> any(Trigger<M>... triggers) {
+    List<Trigger<M>> triggerList = new ArrayList<>();
     for (Trigger trigger : triggers) {
       triggerList.add(trigger);
     }
-    return new AnyTrigger(Collections.unmodifiableList(triggerList));
+    return new AnyTrigger<M>(Collections.unmodifiableList(triggerList));
   }
 
   /**
@@ -98,11 +99,11 @@ public final class Triggers<M extends MessageEnvelope> {
    * <p>Creating a {@link RepeatingTrigger} from an {@link AnyTrigger} is equivalent to creating an {@link AnyTrigger} from
    * its individual {@link RepeatingTrigger}s.
    *
-   * @param <M> the type of input {@link MessageEnvelope} in the window
    * @param trigger the individual trigger to repeat
+   * @param <M> the type of input message in the window
    * @return the created trigger
    */
-  public static <M extends MessageEnvelope> Trigger repeat(Trigger<M> trigger) {
+  public static <M> Trigger<M> repeat(Trigger<M> trigger) {
     return new RepeatingTrigger<>(trigger);
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/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 6aae940..8aa665a 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
@@ -19,18 +19,17 @@
 package org.apache.samza.operators.windows;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.operators.triggers.Trigger;
 
 /**
- * Groups incoming {@link MessageEnvelope}s in the {@link org.apache.samza.operators.MessageStream} into finite
+ * Groups incoming messages in the {@link org.apache.samza.operators.MessageStream} into finite
  * windows for processing.
  *
  * <p> A window is uniquely identified by its {@link WindowKey}. A window can have one or more associated {@link Trigger}s
  * that determine when results from the {@link Window} are emitted.
  *
- * <p> Each emitted result contains one or more {@link MessageEnvelope}s in the window and is called a {@link WindowPane}.
- * A pane can include all {@link MessageEnvelope}s collected for the window so far or only the new {@link MessageEnvelope}s
+ * <p> Each emitted result contains one or more messages in the window and is called a {@link WindowPane}.
+ * A pane can include all messagess collected for the window so far or only the new messages
  * since the last emitted pane. (as determined by the {@link AccumulationMode})
  *
  * <p> A window can have early triggers that allow emitting {@link WindowPane}s speculatively before all data for the window
@@ -66,13 +65,12 @@ import org.apache.samza.operators.triggers.Trigger;
  * <p> Use the {@link Windows} APIs to create various windows and the {@link org.apache.samza.operators.triggers.Triggers}
  * APIs to create triggers.
  *
- * @param <M> the type of the input {@link MessageEnvelope}
- * @param <K> the type of the key in the {@link MessageEnvelope} in this {@link org.apache.samza.operators.MessageStream}.
+ * @param <M> the type of the input message
+ * @param <K> the type of the key in the message in this {@link org.apache.samza.operators.MessageStream}.
  * @param <WV> the type of the value in the {@link WindowPane}.
- * @param <WM> the type of the output.
  */
 @InterfaceStability.Unstable
-public interface Window<M extends MessageEnvelope, K, WV, WM extends WindowPane<K, WV>> {
+public interface Window<M, K, WV> {
 
   /**
    * Set the early triggers for this {@link Window}.
@@ -81,7 +79,7 @@ public interface Window<M extends MessageEnvelope, K, WV, WM extends WindowPane<
    * @param trigger the early trigger
    * @return the {@link Window} function with the early trigger
    */
-  Window<M, K, WV, WM> setEarlyTrigger(Trigger<M> trigger);
+  Window<M, K, WV> setEarlyTrigger(Trigger<M> trigger);
 
   /**
    * Set the late triggers for this {@link Window}.
@@ -90,7 +88,7 @@ public interface Window<M extends MessageEnvelope, K, WV, WM extends WindowPane<
    * @param trigger the late trigger
    * @return the {@link Window} function with the late trigger
    */
-  Window<M, K, WV, WM> setLateTrigger(Trigger<M> trigger);
+  Window<M, K, WV> setLateTrigger(Trigger<M> trigger);
 
   /**
    * Specify how a {@link Window} should process its previously emitted {@link WindowPane}s.
@@ -106,6 +104,6 @@ public interface Window<M extends MessageEnvelope, K, WV, WM extends WindowPane<
    * @param mode the accumulation mode
    * @return the {@link Window} function with the specified {@link AccumulationMode}.
    */
-  Window<M, K, WV, WM> setAccumulationMode(AccumulationMode mode);
+  Window<M, K, WV> setAccumulationMode(AccumulationMode mode);
 
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/main/java/org/apache/samza/operators/windows/WindowKey.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/WindowKey.java b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowKey.java
index 7edf3e1..14bd5ab 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/windows/WindowKey.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowKey.java
@@ -21,7 +21,7 @@ package org.apache.samza.operators.windows;
 /**
  * Key for a {@link WindowPane} emitted from a {@link Window}.
  *
- * @param <K> the type of the key in the incoming {@link org.apache.samza.operators.data.MessageEnvelope}.
+ * @param <K> the type of the key in the incoming message.
  *            Windows that are not keyed have a {@link Void} key type.
  *
  */
@@ -29,18 +29,27 @@ public class WindowKey<K> {
 
   private final  K key;
 
-  private final String windowId;
+  private final String paneId;
 
   public WindowKey(K key, String  windowId) {
     this.key = key;
-    this.windowId = windowId;
+    this.paneId = windowId;
   }
 
   public K getKey() {
     return key;
   }
 
-  public String getWindowId() {
-    return windowId;
+  public String getPaneId() {
+    return paneId;
+  }
+
+  @Override
+  public String toString() {
+    String wndKey = "";
+    if (!(key instanceof Void)) {
+      wndKey = String.format("%s:", key.toString());
+    }
+    return String.format("%s%s", wndKey, paneId);
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/main/java/org/apache/samza/operators/windows/WindowPane.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/WindowPane.java b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowPane.java
index 0388048..3b66bd1 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/windows/WindowPane.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowPane.java
@@ -18,16 +18,13 @@
  */
 package org.apache.samza.operators.windows;
 
-import org.apache.samza.operators.data.MessageEnvelope;
-
-
 /**
  * Specifies the result emitted from a {@link Window}.
  *
  * @param <K>  the type of key in the window pane
  * @param <V>  the type of value in the window pane.
  */
-public final class WindowPane<K, V> implements MessageEnvelope<WindowKey<K>, V> {
+public final class WindowPane<K, V> {
 
   private final WindowKey<K> key;
 
@@ -41,11 +38,11 @@ public final class WindowPane<K, V> implements MessageEnvelope<WindowKey<K>, V>
     this.mode = mode;
   }
 
-  @Override public V getMessage() {
+  public V getMessage() {
     return this.value;
   }
 
-  @Override public WindowKey<K> getKey() {
+  public WindowKey<K> getKey() {
     return this.key;
   }
 


[12/14] samza git commit: SAMZA-1073: top-level fluent API

Posted by ni...@apache.org.
SAMZA-1073: top-level fluent API

`Initial draft of top-level fluent API for operator DAGs

Author: Yi Pan (Data Infrastructure) <ni...@gmail.com>

Reviewers: Xinyu Liu <xi...@linkedin.com>, Jacob Maes <jm...@linkedin.com>, Prateek Maheshwari <pm...@linkedin.com>

Closes #51 from nickpan47/samza-fluent-api-v1 and squashes the following commits:

001be63 [Yi Pan (Data Infrastructure)] SAMZA-1073: Addressing review feedbacks. Change StreamGraphFactory to StreamGraphBuilder.
373048a [Yi Pan (Data Infrastructure)] SAMZA-1073: top-level fluent API `


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

Branch: refs/heads/samza-fluent-api-v1
Commit: c249443b1a5b427bc3fab0303b3069133ec0caad
Parents: 09bf833
Author: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Authored: Thu Feb 16 10:18:09 2017 -0800
Committer: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Committed: Thu Feb 16 10:18:09 2017 -0800

----------------------------------------------------------------------
 .../apache/samza/operators/ContextManager.java  |  47 ++++
 .../apache/samza/operators/MessageStream.java   |  80 +++---
 .../apache/samza/operators/OutputStream.java    |  41 +++
 .../org/apache/samza/operators/StreamGraph.java |  94 +++++++
 .../samza/operators/StreamGraphBuilder.java     |  38 +++
 .../samza/operators/StreamOperatorTask.java     |  51 ----
 .../org/apache/samza/operators/StreamSpec.java  |  46 ++++
 .../data/IncomingSystemMessageEnvelope.java     |  63 -----
 .../operators/data/InputMessageEnvelope.java    |  63 +++++
 .../samza/operators/data/MessageEnvelope.java   |   2 +-
 .../operators/functions/FilterFunction.java     |  13 +-
 .../operators/functions/FlatMapFunction.java    |  15 +-
 .../operators/functions/InitableFunction.java   |  40 +++
 .../samza/operators/functions/JoinFunction.java |  39 ++-
 .../samza/operators/functions/MapFunction.java  |  15 +-
 .../samza/operators/functions/SinkFunction.java |  13 +-
 .../samza/operators/triggers/AnyTrigger.java    |   3 +-
 .../samza/operators/triggers/CountTrigger.java  |   4 +-
 .../operators/triggers/RepeatingTrigger.java    |   4 +-
 .../triggers/TimeSinceFirstMessageTrigger.java  |   3 +-
 .../triggers/TimeSinceLastMessageTrigger.java   |   4 +-
 .../samza/operators/triggers/TimeTrigger.java   |   4 +-
 .../samza/operators/triggers/Trigger.java       |   7 +-
 .../samza/operators/triggers/Triggers.java      |  41 +--
 .../apache/samza/operators/windows/Window.java  |  20 +-
 .../samza/operators/windows/WindowKey.java      |  19 +-
 .../samza/operators/windows/WindowPane.java     |   9 +-
 .../apache/samza/operators/windows/Windows.java | 136 +++++-----
 .../windows/internal/WindowInternal.java        |  14 +-
 .../samza/system/ExecutionEnvironment.java      |  73 ++++++
 .../java/org/apache/samza/task/TaskContext.java |  10 +
 .../data/TestIncomingSystemMessage.java         |   2 +-
 .../operators/windows/TestWindowOutput.java     |  35 ---
 .../samza/operators/windows/TestWindowPane.java |  33 +++
 .../samza/operators/MessageStreamImpl.java      | 146 +++++++----
 .../apache/samza/operators/StreamGraphImpl.java | 260 +++++++++++++++++++
 .../operators/StreamOperatorAdaptorTask.java    | 105 --------
 .../functions/PartialJoinFunction.java          |  56 ++++
 .../samza/operators/impl/OperatorGraph.java     | 164 ++++++++++++
 .../samza/operators/impl/OperatorImpl.java      |  22 +-
 .../samza/operators/impl/OperatorImpls.java     | 124 ---------
 .../operators/impl/PartialJoinOperatorImpl.java |  15 +-
 .../samza/operators/impl/RootOperatorImpl.java  |   7 +-
 .../impl/SessionWindowOperatorImpl.java         |  52 ++++
 .../samza/operators/impl/SinkOperatorImpl.java  |   7 +-
 .../operators/impl/StreamOperatorImpl.java      |  14 +-
 .../operators/impl/WindowOperatorImpl.java      |  11 +-
 .../samza/operators/spec/OperatorSpec.java      |  39 ++-
 .../samza/operators/spec/OperatorSpecs.java     | 161 +++++++++---
 .../operators/spec/PartialJoinOperatorSpec.java |  58 +++--
 .../samza/operators/spec/SinkOperatorSpec.java  |  70 ++++-
 .../operators/spec/StreamOperatorSpec.java      |  58 +++--
 .../operators/spec/WindowOperatorSpec.java      |  41 ++-
 .../samza/operators/spec/WindowState.java       |  16 +-
 .../system/RemoteExecutionEnvironment.java      |  37 +++
 .../system/StandaloneExecutionEnvironment.java  |  50 ++++
 .../apache/samza/task/StreamOperatorTask.java   | 111 ++++++++
 .../samza/example/KeyValueStoreExample.java     | 180 +++++++++++++
 .../samza/example/NoContextStreamExample.java   | 151 +++++++++++
 .../samza/example/OrderShipmentJoinExample.java | 188 ++++++++++++++
 .../samza/example/PageViewCounterExample.java   | 129 +++++++++
 .../samza/example/RepartitionExample.java       | 140 ++++++++++
 .../samza/example/TestBasicStreamGraphs.java    |  99 +++++++
 .../samza/example/TestBroadcastExample.java     | 113 ++++++++
 .../apache/samza/example/TestExampleBase.java   |  46 ++++
 .../apache/samza/example/TestJoinExample.java   | 129 +++++++++
 .../apache/samza/example/TestWindowExample.java |  81 ++++++
 .../apache/samza/operators/BroadcastTask.java   |  96 -------
 .../org/apache/samza/operators/JoinTask.java    |  77 ------
 .../operators/TestFluentStreamAdaptorTask.java  |  85 ------
 .../samza/operators/TestFluentStreamTasks.java  | 112 --------
 .../samza/operators/TestMessageStreamImpl.java  |  65 +++--
 .../operators/TestMessageStreamImplUtil.java    |  26 ++
 .../org/apache/samza/operators/WindowTask.java  |  63 -----
 .../samza/operators/impl/TestOperatorImpls.java | 107 ++++++--
 .../operators/impl/TestSinkOperatorImpl.java    |  11 +-
 .../operators/impl/TestStreamOperatorImpl.java  |  20 +-
 .../samza/operators/spec/TestOperatorSpecs.java |  65 +++--
 78 files changed, 3381 insertions(+), 1307 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/main/java/org/apache/samza/operators/ContextManager.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/ContextManager.java b/samza-api/src/main/java/org/apache/samza/operators/ContextManager.java
new file mode 100644
index 0000000..c3b1cf3
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/ContextManager.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;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.config.Config;
+import org.apache.samza.task.TaskContext;
+
+
+/**
+ * Interface class defining methods to initialize and finalize the context used by the transformation functions.
+ */
+@InterfaceStability.Unstable
+public interface ContextManager {
+  /**
+   * The initialization method to create shared context for the whole task in Samza. Default to NO-OP
+   *
+   * @param config  the configuration object for the task
+   * @param context  the {@link TaskContext} object
+   * @return  User-defined task-wide context object
+   */
+  default TaskContext initTaskContext(Config config, TaskContext context) {
+    return context;
+  }
+
+  /**
+   * The finalize method to allow users to close resource initialized in {@link #initTaskContext} method. Default to NO-OP.
+   *
+   */
+  default void finalizeTaskContext() { }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/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 6a2f95b..adeb4c8 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
@@ -19,7 +19,6 @@
 package org.apache.samza.operators;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.operators.functions.FilterFunction;
 import org.apache.samza.operators.functions.FlatMapFunction;
 import org.apache.samza.operators.functions.JoinFunction;
@@ -29,73 +28,83 @@ import org.apache.samza.operators.windows.Window;
 import org.apache.samza.operators.windows.WindowPane;
 
 import java.util.Collection;
+import java.util.function.Function;
 
 
 /**
- * Represents a stream of {@link MessageEnvelope}s.
+ * Represents a stream of messages.
  * <p>
  * A {@link MessageStream} can be transformed into another {@link MessageStream} by applying the transforms in this API.
  *
- * @param <M>  type of {@link MessageEnvelope}s in this stream
+ * @param <M>  type of messages in this stream
  */
 @InterfaceStability.Unstable
-public interface MessageStream<M extends MessageEnvelope> {
+public interface MessageStream<M> {
 
   /**
-   * Applies the provided 1:1 {@link MapFunction} to {@link MessageEnvelope}s in this {@link MessageStream} and returns the
+   * Applies the provided 1:1 function to messages in this {@link MessageStream} and returns the
    * transformed {@link MessageStream}.
    *
-   * @param mapFn the function to transform a {@link MessageEnvelope} to another {@link MessageEnvelope}
-   * @param <TM> the type of {@link MessageEnvelope}s in the transformed {@link MessageStream}
+   * @param mapFn the function to transform a message to another message
+   * @param <TM> the type of messages in the transformed {@link MessageStream}
    * @return the transformed {@link MessageStream}
    */
-  <TM extends MessageEnvelope> MessageStream<TM> map(MapFunction<M, TM> mapFn);
+  <TM> MessageStream<TM> map(MapFunction<M, TM> mapFn);
 
   /**
-   * Applies the provided 1:n {@link FlatMapFunction} to transform a {@link MessageEnvelope} in this {@link MessageStream}
-   * to n {@link MessageEnvelope}s in the transformed {@link MessageStream}
+   * Applies the provided 1:n function to transform a message in this {@link MessageStream}
+   * to n messages in the transformed {@link MessageStream}
    *
-   * @param flatMapFn the function to transform a {@link MessageEnvelope} to zero or more {@link MessageEnvelope}s
-   * @param <TM> the type of {@link MessageEnvelope}s in the transformed {@link MessageStream}
+   * @param flatMapFn the function to transform a message to zero or more messages
+   * @param <TM> the type of messages in the transformed {@link MessageStream}
    * @return the transformed {@link MessageStream}
    */
-  <TM extends MessageEnvelope> MessageStream<TM> flatMap(FlatMapFunction<M, TM> flatMapFn);
+  <TM> MessageStream<TM> flatMap(FlatMapFunction<M, TM> flatMapFn);
 
   /**
-   * Applies the provided {@link FilterFunction} to {@link MessageEnvelope}s in this {@link MessageStream} and returns the
+   * Applies the provided function to messages in this {@link MessageStream} and returns the
    * transformed {@link MessageStream}.
    * <p>
-   * The {@link FilterFunction} is a predicate which determines whether a {@link MessageEnvelope} in this {@link MessageStream}
+   * The {@link Function} is a predicate which determines whether a message in this {@link MessageStream}
    * should be retained in the transformed {@link MessageStream}.
    *
-   * @param filterFn the predicate to filter {@link MessageEnvelope}s from this {@link MessageStream}
+   * @param filterFn the predicate to filter messages from this {@link MessageStream}
    * @return the transformed {@link MessageStream}
    */
   MessageStream<M> filter(FilterFunction<M> filterFn);
 
   /**
-   * Allows sending {@link MessageEnvelope}s in this {@link MessageStream} to an output
-   * {@link org.apache.samza.system.SystemStream} using the provided {@link SinkFunction}.
+   * Allows sending messages in this {@link MessageStream} to an output using the provided {@link SinkFunction}.
    *
-   * @param sinkFn the function to send {@link MessageEnvelope}s in this stream to output systems
+   * NOTE: the output may not be a {@link org.apache.samza.system.SystemStream}. It can be an external database, etc.
+   *
+   * @param sinkFn  the function to send messages in this stream to output
    */
   void sink(SinkFunction<M> sinkFn);
 
   /**
-   * Groups and processes the {@link MessageEnvelope}s in this {@link MessageStream} according to the provided {@link Window}
+   * Allows sending messages in this {@link MessageStream} to an output {@link MessageStream}.
+   *
+   * NOTE: the {@code stream} has to be a {@link MessageStream}.
+   *
+   * @param stream  the output {@link MessageStream}
+   */
+  void sendTo(OutputStream<M> stream);
+
+  /**
+   * Groups the messages in this {@link MessageStream} according to the provided {@link Window} semantics
    * (e.g. tumbling, sliding or session windows) and returns the transformed {@link MessageStream} of
    * {@link WindowPane}s.
    * <p>
    * Use the {@link org.apache.samza.operators.windows.Windows} helper methods to create the appropriate windows.
    *
-   * @param window the window to group and process {@link MessageEnvelope}s from this {@link MessageStream}
-   * @param <K> the type of key in the {@link MessageEnvelope} in this {@link MessageStream}. If a key is specified,
+   * @param window the window to group and process messages from this {@link MessageStream}
+   * @param <K> the type of key in the message in this {@link MessageStream}. If a key is specified,
    *            panes are emitted per-key.
    * @param <WV> the type of value in the {@link WindowPane} in the transformed {@link MessageStream}
-   * @param <WM> the type of {@link WindowPane} in the transformed {@link MessageStream}
    * @return the transformed {@link MessageStream}
    */
-  <K, WV, WM extends WindowPane<K, WV>> MessageStream<WM> window(Window<M, K, WV, WM> window);
+  <K, WV> MessageStream<WindowPane<K, WV>> window(Window<M, K, WV> window);
 
   /**
    * Joins this {@link MessageStream} with another {@link MessageStream} using the provided pairwise {@link JoinFunction}.
@@ -103,23 +112,32 @@ public interface MessageStream<M extends MessageEnvelope> {
    * We currently only support 2-way joins.
    *
    * @param otherStream the other {@link MessageStream} to be joined with
-   * @param joinFn the function to join {@link MessageEnvelope}s from this and the other {@link MessageStream}
+   * @param joinFn the function to join messages from this and the other {@link MessageStream}
    * @param <K> the type of join key
-   * @param <OM> the type of {@link MessageEnvelope}s in the other stream
-   * @param <RM> the type of {@link MessageEnvelope}s resulting from the {@code joinFn}
+   * @param <OM> the type of messages in the other stream
+   * @param <RM> the type of messages resulting from the {@code joinFn}
    * @return the joined {@link MessageStream}
    */
-  <K, OM extends MessageEnvelope<K, ?>, RM extends MessageEnvelope> MessageStream<RM> join(MessageStream<OM> otherStream,
-      JoinFunction<M, OM, RM> joinFn);
+  <K, OM, RM> MessageStream<RM> join(MessageStream<OM> otherStream, JoinFunction<K, M, OM, RM> joinFn);
 
   /**
    * Merge all {@code otherStreams} with this {@link MessageStream}.
    * <p>
-   * The merging streams must have the same {@link MessageEnvelope} type {@code M}.
+   * The merging streams must have the same messages of type {@code M}.
    *
    * @param otherStreams  other {@link MessageStream}s to be merged with this {@link MessageStream}
    * @return  the merged {@link MessageStream}
    */
   MessageStream<M> merge(Collection<MessageStream<M>> otherStreams);
-  
+
+  /**
+   * Send the input message to an output {@link org.apache.samza.system.SystemStream} and consume it as input {@link MessageStream} again.
+   *
+   * Note: this is an transform function only used in logic DAG. In a physical DAG, this is either translated to a NOOP function, or a {@code MessageStream#sendThrough} function.
+   *
+   * @param parKeyExtractor  a {@link Function} that extract the partition key from a message in this {@link MessageStream}
+   * @param <K>  the type of partition key
+   * @return  a {@link MessageStream} object after the re-partition
+   */
+  <K> MessageStream<M> partitionBy(Function<M, K> parKeyExtractor);
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/main/java/org/apache/samza/operators/OutputStream.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/OutputStream.java b/samza-api/src/main/java/org/apache/samza/operators/OutputStream.java
new file mode 100644
index 0000000..179f0e7
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/OutputStream.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.operators.functions.SinkFunction;
+
+
+/**
+ * The interface class defining the specific {@link SinkFunction} for a system {@link OutputStream}.
+ *
+ * @param <M>  The type of message to be send to this output stream
+ */
+@InterfaceStability.Unstable
+public interface OutputStream<M> {
+
+  /**
+   * Returns the specific {@link SinkFunction} for this output stream. The {@link OutputStream} is created
+   * via {@link StreamGraph#createOutStream(StreamSpec, Serde, Serde)} or {@link StreamGraph#createIntStream(StreamSpec, Serde, Serde)}.
+   * Hence, the proper types of serdes for key and value are instantiated and are used in the {@link SinkFunction} returned.
+   *
+   * @return  The pre-defined {@link SinkFunction} to apply proper serdes before sending the message to the output stream.
+   */
+  SinkFunction<M> getSinkFunction();
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java b/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java
new file mode 100644
index 0000000..abc9861
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/StreamGraph.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.serializers.Serde;
+
+import java.util.Map;
+
+
+/**
+ * Job-level programming interface to create an operator DAG and run in various different runtime environments.
+ */
+@InterfaceStability.Unstable
+public interface StreamGraph {
+  /**
+   * Method to add an input {@link MessageStream} from the system
+   *
+   * @param streamSpec  the {@link StreamSpec} describing the physical characteristics of the input {@link MessageStream}
+   * @param keySerde  the serde used to serialize/deserialize the message key from the input {@link MessageStream}
+   * @param msgSerde  the serde used to serialize/deserialize the message body from the input {@link MessageStream}
+   * @param <K>  the type of key in the input message
+   * @param <V>  the type of message in the input message
+   * @param <M>  the type of {@link MessageEnvelope} in the input {@link MessageStream}
+   * @return   the input {@link MessageStream} object
+   */
+  <K, V, M extends MessageEnvelope<K, V>> MessageStream<M> createInStream(StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde);
+
+  /**
+   * Method to add an output {@link MessageStream} from the system
+   *
+   * @param streamSpec  the {@link StreamSpec} describing the physical characteristics of the output {@link MessageStream}
+   * @param keySerde  the serde used to serialize/deserialize the message key from the output {@link MessageStream}
+   * @param msgSerde  the serde used to serialize/deserialize the message body from the output {@link MessageStream}
+   * @param <K>  the type of key in the output message
+   * @param <V>  the type of message in the output message
+   * @param <M>  the type of {@link MessageEnvelope} in the output {@link MessageStream}
+   * @return   the output {@link MessageStream} object
+   */
+  <K, V, M extends MessageEnvelope<K, V>> OutputStream<M> createOutStream(StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde);
+
+  /**
+   * Method to add an intermediate {@link MessageStream} from the system
+   *
+   * @param streamSpec  the {@link StreamSpec} describing the physical characteristics of the intermediate {@link MessageStream}
+   * @param keySerde  the serde used to serialize/deserialize the message key from the intermediate {@link MessageStream}
+   * @param msgSerde  the serde used to serialize/deserialize the message body from the intermediate {@link MessageStream}
+   * @param <K>  the type of key in the intermediate message
+   * @param <V>  the type of message in the intermediate message
+   * @param <M>  the type of {@link MessageEnvelope} in the intermediate {@link MessageStream}
+   * @return   the intermediate {@link MessageStream} object
+   */
+  <K, V, M extends MessageEnvelope<K, V>> OutputStream<M> createIntStream(StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde);
+
+  /**
+   * Method to get the input {@link MessageStream}s
+   *
+   * @return the input {@link MessageStream}
+   */
+  Map<StreamSpec, MessageStream> getInStreams();
+
+  /**
+   * Method to get the {@link OutputStream}s
+   *
+   * @return  the map of all {@link OutputStream}s
+   */
+  Map<StreamSpec, OutputStream> getOutStreams();
+
+  /**
+   * Method to set the {@link ContextManager} for this {@link StreamGraph}
+   *
+   * @param manager  the {@link ContextManager} object
+   * @return  this {@link StreamGraph} object
+   */
+  StreamGraph withContextManager(ContextManager manager);
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/main/java/org/apache/samza/operators/StreamGraphBuilder.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/StreamGraphBuilder.java b/samza-api/src/main/java/org/apache/samza/operators/StreamGraphBuilder.java
new file mode 100644
index 0000000..b415cf8
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/StreamGraphBuilder.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;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.config.Config;
+
+
+/**
+ * This interface defines a factory class that user will implement to create user-defined operator DAG in a {@link StreamGraph} object.
+ */
+@InterfaceStability.Unstable
+public interface StreamGraphBuilder {
+  /**
+   * Users are required to implement this abstract method to initialize the processing logic of the application, in terms
+   * of a DAG of {@link org.apache.samza.operators.MessageStream}s and operators
+   *
+   * @param graph  an empty {@link StreamGraph} object to be initialized
+   * @param config  the {@link Config} of the application
+   */
+  void init(StreamGraph graph, Config config);
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/main/java/org/apache/samza/operators/StreamOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/StreamOperatorTask.java b/samza-api/src/main/java/org/apache/samza/operators/StreamOperatorTask.java
deleted file mode 100644
index 16cf27a..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/StreamOperatorTask.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators;
-
-import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.IncomingSystemMessageEnvelope;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.util.Map;
-
-
-/**
- * A {@link StreamOperatorTask} is the basic interface to implement for processing {@link MessageStream}s.
- * Implementations can describe the transformation steps for each {@link MessageStream} in the
- * {@link #transform} method using {@link MessageStream} APIs.
- * <p>
- * Implementations may be augmented by implementing {@link org.apache.samza.task.InitableTask},
- * {@link org.apache.samza.task.WindowableTask} and {@link org.apache.samza.task.ClosableTask} interfaces,
- * but should not implement {@link org.apache.samza.task.StreamTask} or {@link org.apache.samza.task.AsyncStreamTask}
- * interfaces.
- */
-@InterfaceStability.Unstable
-public interface StreamOperatorTask {
-
-  /**
-   * Describe the transformation steps for each {@link MessageStream}s for this task using the
-   * {@link MessageStream} APIs. Each {@link MessageStream} corresponds to one {@link SystemStreamPartition}
-   * in the input system.
-   *
-   * @param messageStreams the {@link MessageStream}s that receive {@link IncomingSystemMessageEnvelope}s
-   *                       from their corresponding {@link org.apache.samza.system.SystemStreamPartition}
-   */
-  void transform(Map<SystemStreamPartition, MessageStream<IncomingSystemMessageEnvelope>> messageStreams);
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/main/java/org/apache/samza/operators/StreamSpec.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/StreamSpec.java b/samza-api/src/main/java/org/apache/samza/operators/StreamSpec.java
new file mode 100644
index 0000000..c8a5e8d
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/StreamSpec.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.system.SystemStream;
+
+import java.util.Properties;
+
+
+/**
+ * This interface defines the specification of a {@link SystemStream}. It will be used by the {@link org.apache.samza.system.SystemAdmin}
+ * to create a {@link SystemStream}
+ */
+@InterfaceStability.Unstable
+public interface StreamSpec {
+  /**
+   * Get the {@link SystemStream}
+   *
+   * @return  {@link SystemStream} object
+   */
+  SystemStream getSystemStream();
+
+  /**
+   * Get the physical properties of the {@link SystemStream}
+   *
+   * @return  the properties of this stream
+   */
+  Properties getProperties();
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessageEnvelope.java b/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessageEnvelope.java
deleted file mode 100644
index a65809c..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessageEnvelope.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators.data;
-
-import org.apache.samza.system.IncomingMessageEnvelope;
-import org.apache.samza.system.SystemStreamPartition;
-
-
-/**
- * A {@link MessageEnvelope} that provides additional information about its input {@link SystemStreamPartition}
- * and its {@link Offset} within the {@link SystemStreamPartition}.
- * <p>
- * Note: the {@link Offset} is only unique and comparable within its {@link SystemStreamPartition}.
- */
-public class IncomingSystemMessageEnvelope implements MessageEnvelope<Object, Object> {
-
-  private final IncomingMessageEnvelope ime;
-
-  /**
-   * Creates an {@code IncomingSystemMessageEnvelope} from the {@link IncomingMessageEnvelope}.
-   *
-   * @param ime  the {@link IncomingMessageEnvelope} from the input system.
-   */
-  public IncomingSystemMessageEnvelope(IncomingMessageEnvelope ime) {
-    this.ime = ime;
-  }
-
-  @Override
-  public Object getKey() {
-    return this.ime.getKey();
-  }
-
-  @Override
-  public Object getMessage() {
-    return this.ime.getMessage();
-  }
-
-  public Offset getOffset() {
-    // TODO: need to add offset factory to generate different types of offset. This is just a placeholder,
-    // assuming incoming message envelope carries long value as offset (i.e. Kafka case)
-    return new LongOffset(this.ime.getOffset());
-  }
-
-  public SystemStreamPartition getSystemStreamPartition() {
-    return this.ime.getSystemStreamPartition();
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/main/java/org/apache/samza/operators/data/InputMessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/data/InputMessageEnvelope.java b/samza-api/src/main/java/org/apache/samza/operators/data/InputMessageEnvelope.java
new file mode 100644
index 0000000..306145b
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/data/InputMessageEnvelope.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.data;
+
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.SystemStreamPartition;
+
+
+/**
+ * A {@link MessageEnvelope} that provides additional information about its input {@link SystemStreamPartition}
+ * and its {@link Offset} within the {@link SystemStreamPartition}.
+ * <p>
+ * Note: the {@link Offset} is only unique and comparable within its {@link SystemStreamPartition}.
+ */
+public class InputMessageEnvelope implements MessageEnvelope<Object, Object> {
+
+  private final IncomingMessageEnvelope ime;
+
+  /**
+   * Creates an {@code InputMessageEnvelope} from the {@link IncomingMessageEnvelope}.
+   *
+   * @param ime  the {@link IncomingMessageEnvelope} from the input system.
+   */
+  public InputMessageEnvelope(IncomingMessageEnvelope ime) {
+    this.ime = ime;
+  }
+
+  @Override
+  public Object getKey() {
+    return this.ime.getKey();
+  }
+
+  @Override
+  public Object getMessage() {
+    return this.ime.getMessage();
+  }
+
+  public Offset getOffset() {
+    // TODO: need to add offset factory to generate different types of offset. This is just a placeholder,
+    // assuming incoming message envelope carries long value as offset (i.e. Kafka case)
+    return new LongOffset(this.ime.getOffset());
+  }
+
+  public SystemStreamPartition getSystemStreamPartition() {
+    return this.ime.getSystemStreamPartition();
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/main/java/org/apache/samza/operators/data/MessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/data/MessageEnvelope.java b/samza-api/src/main/java/org/apache/samza/operators/data/MessageEnvelope.java
index ad64231..703a44c 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/data/MessageEnvelope.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/data/MessageEnvelope.java
@@ -23,7 +23,7 @@ import org.apache.samza.annotation.InterfaceStability;
 
 
 /**
- * An entry in the input/output {@link org.apache.samza.operators.MessageStream}s.
+ * An entry in the input/output {@link org.apache.samza.operators.MessageStream}s
  */
 @InterfaceStability.Unstable
 public interface MessageEnvelope<K, M> {

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/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 e611cd0..58479d6 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
@@ -19,21 +19,20 @@
 package org.apache.samza.operators.functions;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.MessageEnvelope;
 
 
 /**
- * A function that specifies whether a {@link MessageEnvelope} should be retained for further processing or filtered out.
- * @param <M>  type of the input {@link MessageEnvelope}
+ * A function that specifies whether a message should be retained for further processing or filtered out.
+ * @param <M>  type of the input message
  */
 @InterfaceStability.Unstable
 @FunctionalInterface
-public interface FilterFunction<M extends MessageEnvelope> {
+public interface FilterFunction<M> extends InitableFunction {
 
   /**
-   * Returns a boolean indicating whether this {@link MessageEnvelope} should be retained or filtered out.
-   * @param message  the {@link MessageEnvelope} to be checked
-   * @return  true if {@link MessageEnvelope} should be retained
+   * Returns a boolean indicating whether this message should be retained or filtered out.
+   * @param message  the input message to be checked
+   * @return  true if {@code message} should be retained
    */
   boolean apply(M message);
 

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/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 dbc0bd9..bbbddeb 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
@@ -19,25 +19,24 @@
 package org.apache.samza.operators.functions;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.MessageEnvelope;
 
 import java.util.Collection;
 
 
 /**
- * A function that transforms a {@link MessageEnvelope} into a collection of 0 or more {@link MessageEnvelope}s,
+ * A function that transforms an input message into a collection of 0 or more messages,
  * possibly of a different type.
- * @param <M>  type of the input {@link MessageEnvelope}
- * @param <OM>  type of the transformed {@link MessageEnvelope}s
+ * @param <M>  type of the input message
+ * @param <OM>  type of the transformed messages
  */
 @InterfaceStability.Unstable
 @FunctionalInterface
-public interface FlatMapFunction<M extends MessageEnvelope, OM extends MessageEnvelope> {
+public interface FlatMapFunction<M, OM>  extends InitableFunction {
 
   /**
-   * Transforms the provided {@link MessageEnvelope} into a collection of 0 or more {@link MessageEnvelope}s.
-   * @param message  the {@link MessageEnvelope} to be transformed
-   * @return  a collection of 0 or more transformed {@link MessageEnvelope}s
+   * Transforms the provided message into a collection of 0 or more messages.
+   * @param message  the input message to be transformed
+   * @return  a collection of 0 or more transformed messages
    */
   Collection<OM> apply(M message);
 

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java
new file mode 100644
index 0000000..2f738da
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.functions;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.config.Config;
+import org.apache.samza.task.TaskContext;
+
+
+/**
+ * interface defined to initalize the context of message transformation functions
+ */
+@InterfaceStability.Unstable
+public interface InitableFunction {
+
+  /**
+   * Interface method to initialize the context for a specific message transformation function.
+   *
+   * @param config  the {@link Config} object for this task
+   * @param context  the {@link TaskContext} object for this task
+   */
+  default void init(Config config, TaskContext context) { }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/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 8cb1fce..fc38177 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
@@ -19,26 +19,41 @@
 package org.apache.samza.operators.functions;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.MessageEnvelope;
 
 
 /**
- * A function that joins {@link MessageEnvelope}s from two {@link org.apache.samza.operators.MessageStream}s and produces
- * a joined {@link MessageEnvelope}.
- * @param <M>  type of the input {@link MessageEnvelope}
- * @param <JM>  type of the {@link MessageEnvelope} to join with
- * @param <RM>  type of the joined {@link MessageEnvelope}
+ * A function that joins messages from two {@link org.apache.samza.operators.MessageStream}s and produces
+ * a joined message.
+ * @param <K>  type of the join key
+ * @param <M>  type of the input message
+ * @param <JM>  type of the message to join with
+ * @param <RM>  type of the joined message
  */
 @InterfaceStability.Unstable
-@FunctionalInterface
-public interface JoinFunction<M extends MessageEnvelope, JM extends MessageEnvelope, RM extends MessageEnvelope> {
+public interface JoinFunction<K, M, JM, RM>  extends InitableFunction {
 
   /**
-   * Join the provided {@link MessageEnvelope}s and produces the joined {@link MessageEnvelope}.
-   * @param message  the input {@link MessageEnvelope}
-   * @param otherMessage  the {@link MessageEnvelope} to join with
-   * @return  the joined {@link MessageEnvelope}
+   * Join the provided input messages and produces the joined messages.
+   * @param message  the input message
+   * @param otherMessage  the message to join with
+   * @return  the joined message
    */
   RM apply(M message, JM otherMessage);
 
+  /**
+   * Method to get the join key in the messages from the first input stream
+   *
+   * @param message  the input message from the first input stream
+   * @return  the join key
+   */
+  K getFirstKey(M message);
+
+  /**
+   * Method to get the join key in the messages from the second input stream
+   *
+   * @param message  the input message from the second input stream
+   * @return  the join key
+   */
+  K getSecondKey(JM message);
+
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/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 04919a7..05a554f 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
@@ -19,22 +19,21 @@
 package org.apache.samza.operators.functions;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.MessageEnvelope;
 
 
 /**
- * A function that transforms a {@link MessageEnvelope} into another {@link MessageEnvelope}, possibly of a different type.
- * @param <M>  type of the input {@link MessageEnvelope}
- * @param <OM>  type of the transformed {@link MessageEnvelope}
+ * A function that transforms an input message into another message, possibly of a different type.
+ * @param <M>  type of the input message
+ * @param <OM>  type of the transformed message
  */
 @InterfaceStability.Unstable
 @FunctionalInterface
-public interface MapFunction<M extends MessageEnvelope, OM extends MessageEnvelope> {
+public interface MapFunction<M, OM>  extends InitableFunction {
 
   /**
-   * Transforms the provided {@link MessageEnvelope} into another {@link MessageEnvelope}
-   * @param message  the {@link MessageEnvelope} to be transformed
-   * @return  the transformed {@link MessageEnvelope}
+   * Transforms the provided message into another message
+   * @param message  the input message to be transformed
+   * @return  the transformed message
    */
   OM apply(M message);
 

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/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 505da92..08e090a 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
@@ -19,26 +19,25 @@
 package org.apache.samza.operators.functions;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.task.MessageCollector;
 import org.apache.samza.task.TaskCoordinator;
 
 
 /**
- * A function that allows sending a {@link MessageEnvelope} to an output system.
- * @param <M>  type of the input {@link MessageEnvelope}
+ * A function that allows sending a message to an output system.
+ * @param <M>  type of the input message
  */
 @InterfaceStability.Unstable
 @FunctionalInterface
-public interface SinkFunction<M extends MessageEnvelope> {
+public interface SinkFunction<M>  extends InitableFunction {
 
   /**
-   * Allows sending the provided {@link MessageEnvelope} to an output {@link org.apache.samza.system.SystemStream} using
+   * Allows sending the provided message to an output {@link org.apache.samza.system.SystemStream} using
    * the provided {@link MessageCollector}. Also provides access to the {@link TaskCoordinator} to request commits
    * or shut the container down.
    *
-   * @param message  the {@link MessageEnvelope} to be sent to an output {@link org.apache.samza.system.SystemStream}
-   * @param messageCollector  the {@link MessageCollector} to use to send the {@link MessageEnvelope}
+   * @param message  the input message to be sent to an output {@link org.apache.samza.system.SystemStream}
+   * @param messageCollector  the {@link MessageCollector} to send the {@link org.apache.samza.operators.data.MessageEnvelope}
    * @param taskCoordinator  the {@link TaskCoordinator} to request commits or shutdown
    */
   void apply(M message, MessageCollector messageCollector, TaskCoordinator taskCoordinator);

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/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 3ca4e9a..6e134df 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,13 +18,12 @@
 */
 package org.apache.samza.operators.triggers;
 
-import org.apache.samza.operators.data.MessageEnvelope;
 import java.util.List;
 
 /**
  * A {@link Trigger} fires as soon as any of its individual triggers has fired.
  */
-public class AnyTrigger<M extends MessageEnvelope> implements Trigger {
+public class AnyTrigger<M> implements Trigger {
 
   private final List<Trigger> triggers;
 

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/main/java/org/apache/samza/operators/triggers/CountTrigger.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/triggers/CountTrigger.java b/samza-api/src/main/java/org/apache/samza/operators/triggers/CountTrigger.java
index ba14928..1cf930c 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/triggers/CountTrigger.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/triggers/CountTrigger.java
@@ -18,13 +18,11 @@
  */
 package org.apache.samza.operators.triggers;
 
-import org.apache.samza.operators.data.MessageEnvelope;
-
 /**
  * A {@link Trigger} that fires when the number of messages in the {@link org.apache.samza.operators.windows.WindowPane}
  * reaches the specified count.
  */
-public class CountTrigger<M extends MessageEnvelope> implements Trigger {
+public class CountTrigger<M> implements Trigger {
 
   private final long count;
 

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/main/java/org/apache/samza/operators/triggers/RepeatingTrigger.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/triggers/RepeatingTrigger.java b/samza-api/src/main/java/org/apache/samza/operators/triggers/RepeatingTrigger.java
index ae9564d..7f78eb8 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/triggers/RepeatingTrigger.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/triggers/RepeatingTrigger.java
@@ -18,12 +18,10 @@
  */
 package org.apache.samza.operators.triggers;
 
-import org.apache.samza.operators.data.MessageEnvelope;
-
 /**
  * A {@link Trigger} that repeats its underlying trigger forever.
  */
-class RepeatingTrigger<M extends MessageEnvelope> implements Trigger<M> {
+class RepeatingTrigger<M> implements Trigger<M> {
 
   private final Trigger<M> trigger;
 

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeSinceFirstMessageTrigger.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeSinceFirstMessageTrigger.java b/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeSinceFirstMessageTrigger.java
index 13fc3cd..4de60a2 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeSinceFirstMessageTrigger.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeSinceFirstMessageTrigger.java
@@ -19,7 +19,6 @@
 
 package org.apache.samza.operators.triggers;
 
-import org.apache.samza.operators.data.MessageEnvelope;
 
 import java.time.Duration;
 
@@ -27,7 +26,7 @@ import java.time.Duration;
  * A {@link Trigger} that fires after the specified duration has passed since the first {@link MessageEnvelope} in
  * the window pane.
  */
-public class TimeSinceFirstMessageTrigger<M extends MessageEnvelope> implements Trigger {
+public class TimeSinceFirstMessageTrigger<M> implements Trigger {
 
   private final Duration duration;
   private final DurationCharacteristic characteristic = DurationCharacteristic.PROCESSING_TIME;

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeSinceLastMessageTrigger.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeSinceLastMessageTrigger.java b/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeSinceLastMessageTrigger.java
index 0150d86..6b09625 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeSinceLastMessageTrigger.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeSinceLastMessageTrigger.java
@@ -18,14 +18,12 @@
  */
 package org.apache.samza.operators.triggers;
 
-import org.apache.samza.operators.data.MessageEnvelope;
-
 import java.time.Duration;
 
 /*
  * A {@link Trigger} that fires when there are no new {@link MessageEnvelope}s in the window pane for the specified duration.
  */
-public class TimeSinceLastMessageTrigger<M extends MessageEnvelope> implements Trigger {
+public class TimeSinceLastMessageTrigger<M> implements Trigger {
 
   private final Duration duration;
   private final DurationCharacteristic characteristic = DurationCharacteristic.PROCESSING_TIME;

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeTrigger.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeTrigger.java b/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeTrigger.java
index ed7fef7..c5875aa 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeTrigger.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/triggers/TimeTrigger.java
@@ -18,14 +18,12 @@
  */
 package org.apache.samza.operators.triggers;
 
-import org.apache.samza.operators.data.MessageEnvelope;
-
 import java.time.Duration;
 
 /*
  * A {@link Trigger} that fires after the specified duration in processing time.
  */
-public class TimeTrigger<M extends MessageEnvelope> implements Trigger {
+public class TimeTrigger<M> implements Trigger {
 
   private final Duration duration;
   private final DurationCharacteristic characteristic = DurationCharacteristic.PROCESSING_TIME;

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/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 6dc4f43..be0a877 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,15 +20,16 @@
 package org.apache.samza.operators.triggers;
 
 
-import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.annotation.InterfaceStability;
 
 /**
  * Marker interface for all triggers. The firing of a trigger indicates the completion of a window pane.
  *
  * <p> Use the {@link Triggers} APIs to create a {@link Trigger}.
  *
- * @param <M> the type of the incoming {@link MessageEnvelope}
+ * @param <M> the type of the incoming message
  */
-public interface Trigger<M extends MessageEnvelope> {
+@InterfaceStability.Unstable
+public interface Trigger<M> {
 
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/main/java/org/apache/samza/operators/triggers/Triggers.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/triggers/Triggers.java b/samza-api/src/main/java/org/apache/samza/operators/triggers/Triggers.java
index f27cfd8..97fb7b7 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/triggers/Triggers.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/triggers/Triggers.java
@@ -19,7 +19,6 @@
 package org.apache.samza.operators.triggers;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.MessageEnvelope;
 
 import java.time.Duration;
 import java.util.ArrayList;
@@ -35,61 +34,63 @@ import java.util.List;
  * <pre> {@code
  *   MessageStream<> windowedStream = stream.window(Windows.tumblingWindow(Duration.of(10, TimeUnit.SECONDS))
  *     .setEarlyTrigger(Triggers.repeat(Triggers.any(Triggers.count(50), Triggers.timeSinceFirstMessage(Duration.of(4, TimeUnit.SECONDS))))))
- *     .accumulateFiredPanes());
+ *     .setAccumulationMode(AccumulationMode.ACCUMULATING));
  * }</pre>
  *
- * @param <M> the type of input {@link MessageEnvelope}s in the {@link org.apache.samza.operators.MessageStream}
  */
 @InterfaceStability.Unstable
-public final class Triggers<M extends MessageEnvelope> {
+public final class Triggers {
 
   private Triggers() { }
 
   /**
-   * Creates a {@link Trigger} that fires when the number of {@link MessageEnvelope}s in the pane
+   * Creates a {@link Trigger} that fires when the number of messages in the pane
    * reaches the specified count.
    *
-   * @param count the number of {@link MessageEnvelope}s to fire the trigger after
+   * @param count the number of messages to fire the trigger after
+   * @param <M> the type of input message in the window
    * @return the created trigger
    */
-  public static Trigger count(long count) {
-    return new CountTrigger(count);
+  public static <M> Trigger<M> count(long count) {
+    return new CountTrigger<M>(count);
   }
 
   /**
-   * Creates a trigger that fires after the specified duration has passed since the first {@link MessageEnvelope} in
+   * Creates a trigger that fires after the specified duration has passed since the first message in
    * the pane.
    *
    * @param duration the duration since the first element
+   * @param <M> the type of input message in the window
    * @return the created trigger
    */
-  public static Trigger timeSinceFirstMessage(Duration duration) {
-    return new TimeSinceFirstMessageTrigger(duration);
+  public static <M> Trigger<M> timeSinceFirstMessage(Duration duration) {
+    return new TimeSinceFirstMessageTrigger<M>(duration);
   }
 
   /**
-   * Creates a trigger that fires when there is no new {@link MessageEnvelope} for the specified duration in the pane.
+   * Creates a trigger that fires when there is no new message for the specified duration in the pane.
    *
    * @param duration the duration since the last element
+   * @param <M> the type of input message in the window
    * @return the created trigger
    */
-  public static Trigger timeSinceLastMessage(Duration duration) {
-    return new TimeSinceLastMessageTrigger(duration);
+  public static <M> Trigger<M> timeSinceLastMessage(Duration duration) {
+    return new TimeSinceLastMessageTrigger<M>(duration);
   }
 
   /**
    * Creates a trigger that fires when any of the provided triggers fire.
    *
-   * @param <M> the type of input {@link MessageEnvelope} in the window
    * @param triggers the individual triggers
+   * @param <M> the type of input message in the window
    * @return the created trigger
    */
-  public static <M extends MessageEnvelope> Trigger any(Trigger<M>... triggers) {
-    List<Trigger> triggerList = new ArrayList<>();
+  public static <M> Trigger<M> any(Trigger<M>... triggers) {
+    List<Trigger<M>> triggerList = new ArrayList<>();
     for (Trigger trigger : triggers) {
       triggerList.add(trigger);
     }
-    return new AnyTrigger(Collections.unmodifiableList(triggerList));
+    return new AnyTrigger<M>(Collections.unmodifiableList(triggerList));
   }
 
   /**
@@ -98,11 +99,11 @@ public final class Triggers<M extends MessageEnvelope> {
    * <p>Creating a {@link RepeatingTrigger} from an {@link AnyTrigger} is equivalent to creating an {@link AnyTrigger} from
    * its individual {@link RepeatingTrigger}s.
    *
-   * @param <M> the type of input {@link MessageEnvelope} in the window
    * @param trigger the individual trigger to repeat
+   * @param <M> the type of input message in the window
    * @return the created trigger
    */
-  public static <M extends MessageEnvelope> Trigger repeat(Trigger<M> trigger) {
+  public static <M> Trigger<M> repeat(Trigger<M> trigger) {
     return new RepeatingTrigger<>(trigger);
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/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 6aae940..9609292 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
@@ -19,18 +19,17 @@
 package org.apache.samza.operators.windows;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.operators.triggers.Trigger;
 
 /**
- * Groups incoming {@link MessageEnvelope}s in the {@link org.apache.samza.operators.MessageStream} into finite
+ * Groups incoming messages in the {@link org.apache.samza.operators.MessageStream} into finite
  * windows for processing.
  *
  * <p> A window is uniquely identified by its {@link WindowKey}. A window can have one or more associated {@link Trigger}s
  * that determine when results from the {@link Window} are emitted.
  *
- * <p> Each emitted result contains one or more {@link MessageEnvelope}s in the window and is called a {@link WindowPane}.
- * A pane can include all {@link MessageEnvelope}s collected for the window so far or only the new {@link MessageEnvelope}s
+ * <p> Each emitted result contains one or more messages in the window and is called a {@link WindowPane}.
+ * A pane can include all messages collected for the window so far or only the new messages
  * since the last emitted pane. (as determined by the {@link AccumulationMode})
  *
  * <p> A window can have early triggers that allow emitting {@link WindowPane}s speculatively before all data for the window
@@ -66,13 +65,12 @@ import org.apache.samza.operators.triggers.Trigger;
  * <p> Use the {@link Windows} APIs to create various windows and the {@link org.apache.samza.operators.triggers.Triggers}
  * APIs to create triggers.
  *
- * @param <M> the type of the input {@link MessageEnvelope}
- * @param <K> the type of the key in the {@link MessageEnvelope} in this {@link org.apache.samza.operators.MessageStream}.
+ * @param <M> the type of the input message
+ * @param <K> the type of the key in the message in this {@link org.apache.samza.operators.MessageStream}.
  * @param <WV> the type of the value in the {@link WindowPane}.
- * @param <WM> the type of the output.
  */
 @InterfaceStability.Unstable
-public interface Window<M extends MessageEnvelope, K, WV, WM extends WindowPane<K, WV>> {
+public interface Window<M, K, WV> {
 
   /**
    * Set the early triggers for this {@link Window}.
@@ -81,7 +79,7 @@ public interface Window<M extends MessageEnvelope, K, WV, WM extends WindowPane<
    * @param trigger the early trigger
    * @return the {@link Window} function with the early trigger
    */
-  Window<M, K, WV, WM> setEarlyTrigger(Trigger<M> trigger);
+  Window<M, K, WV> setEarlyTrigger(Trigger<M> trigger);
 
   /**
    * Set the late triggers for this {@link Window}.
@@ -90,7 +88,7 @@ public interface Window<M extends MessageEnvelope, K, WV, WM extends WindowPane<
    * @param trigger the late trigger
    * @return the {@link Window} function with the late trigger
    */
-  Window<M, K, WV, WM> setLateTrigger(Trigger<M> trigger);
+  Window<M, K, WV> setLateTrigger(Trigger<M> trigger);
 
   /**
    * Specify how a {@link Window} should process its previously emitted {@link WindowPane}s.
@@ -106,6 +104,6 @@ public interface Window<M extends MessageEnvelope, K, WV, WM extends WindowPane<
    * @param mode the accumulation mode
    * @return the {@link Window} function with the specified {@link AccumulationMode}.
    */
-  Window<M, K, WV, WM> setAccumulationMode(AccumulationMode mode);
+  Window<M, K, WV> setAccumulationMode(AccumulationMode mode);
 
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowKey.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/WindowKey.java b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowKey.java
index 7edf3e1..14bd5ab 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/windows/WindowKey.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowKey.java
@@ -21,7 +21,7 @@ package org.apache.samza.operators.windows;
 /**
  * Key for a {@link WindowPane} emitted from a {@link Window}.
  *
- * @param <K> the type of the key in the incoming {@link org.apache.samza.operators.data.MessageEnvelope}.
+ * @param <K> the type of the key in the incoming message.
  *            Windows that are not keyed have a {@link Void} key type.
  *
  */
@@ -29,18 +29,27 @@ public class WindowKey<K> {
 
   private final  K key;
 
-  private final String windowId;
+  private final String paneId;
 
   public WindowKey(K key, String  windowId) {
     this.key = key;
-    this.windowId = windowId;
+    this.paneId = windowId;
   }
 
   public K getKey() {
     return key;
   }
 
-  public String getWindowId() {
-    return windowId;
+  public String getPaneId() {
+    return paneId;
+  }
+
+  @Override
+  public String toString() {
+    String wndKey = "";
+    if (!(key instanceof Void)) {
+      wndKey = String.format("%s:", key.toString());
+    }
+    return String.format("%s%s", wndKey, paneId);
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowPane.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/WindowPane.java b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowPane.java
index 0388048..3b66bd1 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/windows/WindowPane.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowPane.java
@@ -18,16 +18,13 @@
  */
 package org.apache.samza.operators.windows;
 
-import org.apache.samza.operators.data.MessageEnvelope;
-
-
 /**
  * Specifies the result emitted from a {@link Window}.
  *
  * @param <K>  the type of key in the window pane
  * @param <V>  the type of value in the window pane.
  */
-public final class WindowPane<K, V> implements MessageEnvelope<WindowKey<K>, V> {
+public final class WindowPane<K, V> {
 
   private final WindowKey<K> key;
 
@@ -41,11 +38,11 @@ public final class WindowPane<K, V> implements MessageEnvelope<WindowKey<K>, V>
     this.mode = mode;
   }
 
-  @Override public V getMessage() {
+  public V getMessage() {
     return this.value;
   }
 
-  @Override public WindowKey<K> getKey() {
+  public WindowKey<K> getKey() {
     return this.key;
   }
 


[13/14] samza git commit: Merge branch 'master' into samza-fluent-api-v1

Posted by ni...@apache.org.
Merge branch 'master' into samza-fluent-api-v1


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

Branch: refs/heads/samza-fluent-api-v1
Commit: 55d7988b924dbfc91c3bc7355d9e9ade0470b250
Parents: 4bde68b c249443
Author: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Authored: Thu Feb 16 10:37:42 2017 -0800
Committer: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Committed: Thu Feb 16 10:37:42 2017 -0800

----------------------------------------------------------------------
 .../samza/zk/ScheduleAfterDebounceTime.java     |  86 +++++++++++++++
 .../samza/zk/TestScheduleAfterDebounceTime.java | 110 +++++++++++++++++++
 2 files changed, 196 insertions(+)
----------------------------------------------------------------------



[11/14] samza git commit: SAMZA-1073: top-level fluent API

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/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 f7e1f36..73fb5c8 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
@@ -20,7 +20,6 @@
 package org.apache.samza.operators.windows;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.operators.triggers.TimeTrigger;
 import org.apache.samza.operators.triggers.Trigger;
 import org.apache.samza.operators.triggers.Triggers;
@@ -34,11 +33,11 @@ import java.util.function.Function;
 /**
  * APIs for creating different types of {@link Window}s.
  *
- * Groups the incoming {@link MessageEnvelope}s in the {@link org.apache.samza.operators.MessageStream} into finite windows for processing.
+ * Groups the incoming messages in the {@link org.apache.samza.operators.MessageStream} into finite windows for processing.
  *
  * <p> Each window is uniquely identified by its {@link WindowKey}. A window can have one or more associated {@link Trigger}s
  * that determine when results from the {@link Window} are emitted. Each emitted result contains one or more
- * {@link MessageEnvelope}s in the window and is called a {@link WindowPane}.
+ * messages in the window and is called a {@link WindowPane}.
  *
  * <p> A window can have early triggers that allow emitting {@link WindowPane}s speculatively before all data for the window
  * has arrived or late triggers that allow handling of late data arrivals.
@@ -74,14 +73,14 @@ import java.util.function.Function;
  *   <li>
  *     Session Windows: A session window groups a {@link org.apache.samza.operators.MessageStream} into sessions.
  *     A <i>session</i> captures some period of activity over a {@link org.apache.samza.operators.MessageStream}.
- *     The boundary for a session is defined by a {@code sessionGap}. All {@link MessageEnvelope}s that that arrive within
+ *     The boundary for a session is defined by a {@code sessionGap}. All messages that that arrive within
  *     the gap are grouped into the same session.
  *   <li>
  *     Global Windows: A global window defines a single infinite window over the entire {@link org.apache.samza.operators.MessageStream}.
  *     An early trigger must be specified when defining a global window.
  * </ul>
  *
- * <p> A {@link Window} is defined as "keyed" when the incoming {@link MessageEnvelope}s are first grouped based on their key
+ * <p> A {@link Window} is defined as "keyed" when the incoming messages are first grouped based on their key
  * and triggers are fired and window panes are emitted per-key. It is possible to construct "keyed" variants of all the above window
  * types.
  *
@@ -92,7 +91,7 @@ public final class Windows {
   private Windows() { }
 
   /**
-   * Creates a {@link Window} that groups incoming {@link MessageEnvelope}s into fixed-size, non-overlapping processing
+   * Creates a {@link Window} that groups incoming messages into fixed-size, non-overlapping processing
    * time based windows based on the provided keyFn and applies the provided fold function to them.
    *
    * <p>The below example computes the maximum value per-key over fixed size 10 second windows.
@@ -101,29 +100,29 @@ public final class Windows {
    *    MessageStream<UserClick> stream = ...;
    *    Function<UserClick, String> keyFn = ...;
    *    BiFunction<UserClick, Integer, Integer> maxAggregator = (m, c)-> Math.max(parseInt(m), c);
-   *    MessageStream<WindowOutput<WindowKey<String>, Integer>> windowedStream = stream.window(
-   *    Windows.keyedTumblingWindow(keyFn, Duration.ofSeconds(10), maxAggregator));
+   *    MessageStream<WindowPane<String, Integer>> windowedStream = stream.window(
+   *        Windows.keyedTumblingWindow(keyFn, Duration.ofSeconds(10), maxAggregator));
    * }
    * </pre>
    *
-   * @param keyFn the function to extract the window key from a {@link MessageEnvelope}
+   * @param keyFn the function to extract the window key from a message
    * @param interval the duration in processing time
-   * @param foldFn the function to aggregate {@link MessageEnvelope}s in the {@link WindowPane}
-   * @param <M> the type of the input {@link MessageEnvelope}
+   * @param foldFn the function to aggregate messages in the {@link WindowPane}
+   * @param <M> the type of the input message
    * @param <WV> the type of the {@link WindowPane} output value
    * @param <K> the type of the key in the {@link Window}
    * @return the created {@link Window} function.
    */
-  public static <M extends MessageEnvelope, K, WV> Window<M, K, WV, WindowPane<K, WV>>
+  public static <M, K, WV> Window<M, K, WV>
     keyedTumblingWindow(Function<M, K> keyFn, Duration interval, BiFunction<M, WV, WV> foldFn) {
 
-    Trigger defaultTrigger = new TimeTrigger(interval);
+    Trigger<M> defaultTrigger = new TimeTrigger<>(interval);
     return new WindowInternal<M, K, WV>(defaultTrigger, foldFn, keyFn, null);
   }
 
 
   /**
-   * Creates a {@link Window} that groups incoming {@link MessageEnvelope}s into fixed-size, non-overlapping
+   * Creates a {@link Window} that groups incoming messages into fixed-size, non-overlapping
    * processing time based windows using the provided keyFn.
    *
    * <p>The below example groups the stream into fixed-size 10 second windows for each key.
@@ -131,19 +130,18 @@ public final class Windows {
    * <pre> {@code
    *    MessageStream<UserClick> stream = ...;
    *    Function<UserClick, String> keyFn = ...;
-   *    MessageStream<WindowOutput<WindowKey<String>, Collection<UserClick>>> windowedStream = stream.window(
-   *    Windows.keyedTumblingWindow(keyFn, Duration.ofSeconds(10)));
+   *    MessageStream<WindowPane<String, Collection<UserClick>>> windowedStream = stream.window(
+   *        Windows.keyedTumblingWindow(keyFn, Duration.ofSeconds(10)));
    * }
    * </pre>
    *
-   * @param keyFn function to extract key from the {@link MessageEnvelope}
+   * @param keyFn function to extract key from the message
    * @param interval the duration in processing time
-   * @param <M> the type of the input {@link MessageEnvelope}
+   * @param <M> the type of the input message
    * @param <K> the type of the key in the {@link Window}
    * @return the created {@link Window} function
    */
-  public static <M extends MessageEnvelope, K> Window<M, K, Collection<M>, WindowPane<K, Collection<M>>>
-    keyedTumblingWindow(Function<M, K> keyFn, Duration interval) {
+  public static <M, K> Window<M, K, Collection<M>> keyedTumblingWindow(Function<M, K> keyFn, Duration interval) {
     BiFunction<M, Collection<M>, Collection<M>> aggregator = (m, c) -> {
       c.add(m);
       return c;
@@ -160,25 +158,25 @@ public final class Windows {
    * <pre> {@code
    *    MessageStream<String> stream = ...;
    *    BiFunction<String, Integer, Integer> maxAggregator = (m, c)-> Math.max(parseInt(m), c);
-   *    MessageStream<WindowOutput<WindowKey, Integer>> windowedStream = stream.window(
-   *    Windows.tumblingWindow(Duration.ofSeconds(10), maxAggregator));
+   *    MessageStream<WindowPane<Void, Integer>> windowedStream = stream.window(
+   *        Windows.tumblingWindow(Duration.ofSeconds(10), maxAggregator));
    * }
    * </pre>
    *
    * @param duration the duration in processing time
-   * @param foldFn to aggregate {@link MessageEnvelope}s in the {@link WindowPane}
-   * @param <M> the type of the input {@link MessageEnvelope}
+   * @param foldFn to aggregate messages in the {@link WindowPane}
+   * @param <M> the type of the input message
    * @param <WV> the type of the {@link WindowPane} output value
    * @return the created {@link Window} function
    */
-  public static <M extends MessageEnvelope, WV> Window<M, Void, WV, WindowPane<Void, WV>>
+  public static <M, WV> Window<M, Void, WV>
     tumblingWindow(Duration duration, BiFunction<M, WV, WV> foldFn) {
-    Trigger defaultTrigger = Triggers.repeat(new TimeTrigger(duration));
-    return new WindowInternal<M, Void, WV>(defaultTrigger, foldFn, null, null);
+    Trigger<M> defaultTrigger = Triggers.repeat(new TimeTrigger<>(duration));
+    return new WindowInternal<>(defaultTrigger, foldFn, null, null);
   }
 
   /**
-   * Creates a {@link Window} that groups incoming {@link MessageEnvelope}s into fixed-size, non-overlapping
+   * Creates a {@link Window} that groups incoming messages into fixed-size, non-overlapping
    * processing time based windows.
    *
    * <p>The below example groups the stream into fixed-size 10 second windows and computes a windowed-percentile.
@@ -187,16 +185,16 @@ public final class Windows {
    *    MessageStream<Long> stream = ...;
    *    Function<Collection<Long, Long>> percentile99 = ..
    *
-   *    MessageStream<WindowOutput<WindowKey, Collection<Long>>> windowedStream = integerStream.window(Windows.tumblingWindow(Duration.ofSeconds(10)));
+   *    MessageStream<WindowPane<Void, Collection<Long>>> windowedStream = integerStream.window(Windows.tumblingWindow(Duration.ofSeconds(10)));
    *    MessageStream<Long> windowedPercentiles = windowed.map(windowedOutput -> percentile99(windowedOutput.getMessage());
    * }
    * </pre>
    *
    * @param duration the duration in processing time
-   * @param <M> the type of the input {@link MessageEnvelope}
+   * @param <M> the type of the input message
    * @return the created {@link Window} function
    */
-  public static <M extends MessageEnvelope> Window<M, Void, Collection<M>, WindowPane<Void, Collection<M>>> tumblingWindow(Duration duration) {
+  public static <M> Window<M, Void, Collection<M>> tumblingWindow(Duration duration) {
     BiFunction<M, Collection<M>, Collection<M>> aggregator = (m, c) -> {
       c.add(m);
       return c;
@@ -205,11 +203,11 @@ public final class Windows {
   }
 
   /**
-   * Creates a {@link Window} that groups incoming {@link MessageEnvelope}s into sessions per-key based on the provided {@code sessionGap}
+   * Creates a {@link Window} that groups incoming messages into sessions per-key based on the provided {@code sessionGap}
    * and applies the provided fold function to them.
    *
    * <p>A <i>session</i> captures some period of activity over a {@link org.apache.samza.operators.MessageStream}.
-   * A session is considered complete when no new messages arrive within the {@code sessionGap}. All {@link MessageEnvelope}s that arrive within
+   * A session is considered complete when no new messages arrive within the {@code sessionGap}. All messages that arrive within
    * the gap are grouped into the same session.
    *
    * <p>The below example computes the maximum value per-key over a session window of gap 10 seconds.
@@ -218,29 +216,29 @@ public final class Windows {
    *    MessageStream<UserClick> stream = ...;
    *    BiFunction<UserClick, Integer, Integer> maxAggregator = (m, c)-> Math.max(parseInt(m), c);
    *    Function<UserClick, String> userIdExtractor = m -> m.getUserId()..;
-   *    MessageStream<WindowOutput<WindowKey<String>, Integer>> windowedStream = stream.window(
-   *    Windows.keyedSessionWindow(userIdExtractor, Duration.minute(1), maxAggregator));
+   *    MessageStream<WindowPane<String, Integer>> windowedStream = stream.window(
+   *        Windows.keyedSessionWindow(userIdExtractor, Duration.minute(1), maxAggregator));
    * }
    * </pre>
    *
-   * @param keyFn the function to extract the window key from a {@link MessageEnvelope}
+   * @param keyFn the function to extract the window key from a message
    * @param sessionGap the timeout gap for defining the session
-   * @param foldFn the function to aggregate {@link MessageEnvelope}s in the {@link WindowPane}
-   * @param <M> the type of the input {@link MessageEnvelope}
+   * @param foldFn the function to aggregate messages in the {@link WindowPane}
+   * @param <M> the type of the input message
    * @param <K> the type of the key in the {@link Window}
    * @param <WV> the type of the output value in the {@link WindowPane}
    * @return the created {@link Window} function
    */
-  public static <M extends MessageEnvelope, K, WV> Window<M, K, WV, WindowPane<K, WV>> keyedSessionWindow(Function<M, K> keyFn, Duration sessionGap, BiFunction<M, WV, WV> foldFn) {
-    Trigger defaultTrigger = Triggers.timeSinceLastMessage(sessionGap);
-    return new WindowInternal<M, K, WV>(defaultTrigger, foldFn, keyFn, null);
+  public static <M, K, WV> Window<M, K, WV> keyedSessionWindow(Function<M, K> keyFn, Duration sessionGap, BiFunction<M, WV, WV> foldFn) {
+    Trigger<M> defaultTrigger = Triggers.timeSinceLastMessage(sessionGap);
+    return new WindowInternal<>(defaultTrigger, foldFn, keyFn, null);
   }
 
   /**
-   * Creates a {@link Window} that groups incoming {@link MessageEnvelope}s into sessions per-key based on the provided {@code sessionGap}.
+   * Creates a {@link Window} that groups incoming messages into sessions per-key based on the provided {@code sessionGap}.
    *
    * <p>A <i>session</i> captures some period of activity over a {@link org.apache.samza.operators.MessageStream}. The
-   * boundary for the session is defined by a {@code sessionGap}. All {@link MessageEnvelope}s that that arrive within
+   * boundary for the session is defined by a {@code sessionGap}. All messages that that arrive within
    * the gap are grouped into the same session.
    *
    * <p>The below example groups the stream into per-key session windows of gap 10 seconds.
@@ -249,18 +247,18 @@ public final class Windows {
    *    MessageStream<UserClick> stream = ...;
    *    BiFunction<UserClick, Integer, Integer> maxAggregator = (m, c)-> Math.max(parseIntField(m), c);
    *    Function<UserClick, String> userIdExtractor = m -> m.getUserId()..;
-   *    MessageStream<WindowOutput<WindowKey<String>, Collection<M>>> windowedStream = stream.window(
-   *    Windows.keyedSessionWindow(userIdExtractor, Duration.ofSeconds(10)));
+   *    MessageStream<WindowPane<String>, Collection<M>> windowedStream = stream.window(
+   *        Windows.keyedSessionWindow(userIdExtractor, Duration.ofSeconds(10)));
    * }
    * </pre>
    *
-   * @param keyFn the function to extract the window key from a {@link MessageEnvelope}
+   * @param keyFn the function to extract the window key from a message}
    * @param sessionGap the timeout gap for defining the session
-   * @param <M> the type of the input {@link MessageEnvelope}
+   * @param <M> the type of the input message
    * @param <K> the type of the key in the {@link Window}
    * @return the created {@link Window} function
    */
-  public static <M extends MessageEnvelope, K> Window<M, K, Collection<M>, WindowPane<K, Collection<M>>> keyedSessionWindow(Function<M, K> keyFn, Duration sessionGap) {
+  public static <M, K> Window<M, K, Collection<M>> keyedSessionWindow(Function<M, K> keyFn, Duration sessionGap) {
 
     BiFunction<M, Collection<M>, Collection<M>> aggregator = (m, c) -> {
       c.add(m);
@@ -271,7 +269,7 @@ public final class Windows {
 
 
   /**
-   * Creates a {@link Window} that groups incoming {@link MessageEnvelope}s into a single global window. This window does not have a
+   * Creates a {@link Window} that groups incoming messages into a single global window. This window does not have a
    * default trigger. The triggering behavior must be specified by setting an early trigger.
    *
    * <p>The below example computes the maximum value over a count based window. The window emits {@link WindowPane}s when
@@ -280,36 +278,36 @@ public final class Windows {
    * <pre> {@code
    *    MessageStream<Long> stream = ...;
    *    BiFunction<Long, Long, Long> maxAggregator = (m, c)-> Math.max(m, c);
-   *    MessageStream<WindowOutput<WindowKey, Long>> windowedStream = stream.window(Windows.globalWindow(maxAggregator)
+   *    MessageStream<WindowPane<Void, Long>> windowedStream = stream.window(Windows.globalWindow(maxAggregator)
    *      .setEarlyTriggers(Triggers.repeat(Triggers.any(Triggers.count(50), Triggers.timeSinceFirstMessage(Duration.ofSeconds(10))))))
    * }
    * </pre>
    *
-   * @param foldFn the function to aggregate {@link MessageEnvelope}s in the {@link WindowPane}
-   * @param <M> the type of {@link MessageEnvelope}
+   * @param foldFn the function to aggregate messages in the {@link WindowPane}
+   * @param <M> the type of message
    * @param <WV> type of the output value in the {@link WindowPane}
    * @return the created {@link Window} function.
    */
-  public static <M extends MessageEnvelope, WV> Window<M, Void, WV, WindowPane<Void, WV>> globalWindow(BiFunction<M, WV, WV> foldFn) {
-    return new WindowInternal<M, Void, WV>(null, foldFn, null, null);
+  public static <M, WV> Window<M, Void, WV> globalWindow(BiFunction<M, WV, WV> foldFn) {
+    return new WindowInternal<>(null, foldFn, null, null);
   }
 
   /**
-   * Creates a {@link Window} that groups incoming {@link MessageEnvelope}s into a single global window. This window does not have a
+   * Creates a {@link Window} that groups incoming messages into a single global window. This window does not have a
    * default trigger. The triggering behavior must be specified by setting an early trigger.
    *
    * The below example groups the stream into count based windows that trigger every 50 messages or every 10 minutes.
    * <pre> {@code
    *    MessageStream<Long> stream = ...;
-   *    MessageStream<WindowOutput<WindowKey, Collection<Long>>> windowedStream = stream.window(Windows.globalWindow()
+   *    MessageStream<WindowPane<Void, Collection<Long>> windowedStream = stream.window(Windows.globalWindow()
    *      .setEarlyTrigger(Triggers.repeat(Triggers.any(Triggers.count(50), Triggers.timeSinceFirstMessage(Duration.ofSeconds(10))))))
    * }
    * </pre>
    *
-   * @param <M> the type of {@link MessageEnvelope}
+   * @param <M> the type of message
    * @return the created {@link Window} function.
    */
-  public static <M extends MessageEnvelope> Window<M, Void, Collection<M>, WindowPane<Void, Collection<M>>> globalWindow() {
+  public static <M> Window<M, Void, Collection<M>> globalWindow() {
     BiFunction<M, Collection<M>, Collection<M>> aggregator = (m, c) -> {
       c.add(m);
       return c;
@@ -318,7 +316,7 @@ public final class Windows {
   }
 
   /**
-   * Returns a global {@link Window} that groups incoming {@link MessageEnvelope}s using the provided keyFn.
+   * Returns a global {@link Window} that groups incoming messages using the provided keyFn.
    * The window does not have a default trigger. The triggering behavior must be specified by setting an early
    * trigger.
    *
@@ -329,24 +327,24 @@ public final class Windows {
    *    MessageStream<UserClick> stream = ...;
    *    BiFunction<UserClick, Long, Long> maxAggregator = (m, c)-> Math.max(parseLongField(m), c);
    *    Function<UserClick, String> keyFn = ...;
-   *    MessageStream<WindowOutput<WindowKey<String>, Long>> windowedStream = stream.window(Windows.keyedGlobalWindow(keyFn, maxAggregator)
+   *    MessageStream<WindowPane<String, Long>> windowedStream = stream.window(Windows.keyedGlobalWindow(keyFn, maxAggregator)
    *      .setEarlyTrigger(Triggers.repeat(Triggers.any(Triggers.count(50), Triggers.timeSinceFirstMessage(Duration.minutes(10))))))
    * }
    * </pre>
    *
-   * @param keyFn the function to extract the window key from a {@link MessageEnvelope}
-   * @param foldFn the function to aggregate {@link MessageEnvelope}s in the {@link WindowPane}
-   * @param <M> the type of {@link MessageEnvelope}
+   * @param keyFn the function to extract the window key from a message
+   * @param foldFn the function to aggregate messages in the {@link WindowPane}
+   * @param <M> the type of message
    * @param <K> type of the key in the {@link Window}
    * @param <WV> the type of the output value in the {@link WindowPane}
    * @return the created {@link Window} function
    */
-  public static <M extends MessageEnvelope, K, WV> Window<M, K, WV, WindowPane<K, WV>> keyedGlobalWindow(Function<M, K> keyFn, BiFunction<M, WV, WV> foldFn) {
+  public static <M, K, WV> Window<M, K, WV> keyedGlobalWindow(Function<M, K> keyFn, BiFunction<M, WV, WV> foldFn) {
     return new WindowInternal<M, K, WV>(null, foldFn, keyFn, null);
   }
 
   /**
-   * Returns a global {@link Window} that groups incoming {@link MessageEnvelope}s using the provided keyFn.
+   * Returns a global {@link Window} that groups incoming messages using the provided keyFn.
    * The window does not have a default trigger. The triggering behavior must be specified by setting an early trigger.
    *
    * <p> The below example groups the stream per-key into count based windows. The window triggers every 50 messages or
@@ -355,17 +353,17 @@ public final class Windows {
    * <pre> {@code
    *    MessageStream<UserClick> stream = ...;
    *    Function<UserClick, String> keyFn = ...;
-   *    MessageStream<WindowOutput<WindowKey<String>, Collection<UserClick>>> windowedStream = stream.window(Windows.keyedGlobalWindow(keyFn)
+   *    MessageStream<WindowPane<String, Collection<UserClick>> windowedStream = stream.window(Windows.keyedGlobalWindow(keyFn)
    *      .setEarlyTrigger(Triggers.repeat(Triggers.any(Triggers.count(50), Triggers.timeSinceFirstMessage(Duration.minutes(10))))))
    * }
    * </pre>
    *
-   * @param keyFn the function to extract the window key from a {@link MessageEnvelope}
-   * @param <M> the type of {@link MessageEnvelope}
+   * @param keyFn the function to extract the window key from a message
+   * @param <M> the type of message
    * @param <K> the type of the key in the {@link Window}
    * @return the created {@link Window} function
    */
-  public static <M extends MessageEnvelope, K> Window<M, K, Collection<M>, WindowPane<K, Collection<M>>> keyedGlobalWindow(Function<M, K> keyFn) {
+  public static <M, K> Window<M, K, Collection<M>> keyedGlobalWindow(Function<M, K> keyFn) {
     BiFunction<M, Collection<M>, Collection<M>> aggregator = (m, c) -> {
       c.add(m);
       return c;

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/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 8825867..9479eea 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,11 +18,9 @@
  */
 package org.apache.samza.operators.windows.internal;
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.MessageEnvelope;
 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.operators.windows.WindowPane;
 
 import java.util.function.BiFunction;
 import java.util.function.Function;
@@ -32,9 +30,13 @@ import java.util.function.Function;
  *  and whether to accumulate or discard previously emitted panes.
  *
  *  Note: This class is meant to be used internally by Samza, and is not to be instantiated by programmers.
+ *
+ * @param <M>  the type of input message
+ * @param <K>  the type of key for the window
+ * @param <WV>  the type of aggregated value in the window output
  */
 @InterfaceStability.Unstable
-public final class WindowInternal<M extends MessageEnvelope, K, WV> implements Window<M, K, WV, WindowPane<K, WV>> {
+public final class WindowInternal<M, K, WV> implements Window<M, K, WV> {
 
   private final Trigger defaultTrigger;
 
@@ -67,19 +69,19 @@ public final class WindowInternal<M extends MessageEnvelope, K, WV> implements W
   }
 
   @Override
-  public Window<M, K, WV, WindowPane<K, WV>> setEarlyTrigger(Trigger trigger) {
+  public Window<M, K, WV> setEarlyTrigger(Trigger trigger) {
     this.earlyTrigger = trigger;
     return this;
   }
 
   @Override
-  public Window<M, K, WV, WindowPane<K, WV>> setLateTrigger(Trigger trigger) {
+  public Window<M, K, WV> setLateTrigger(Trigger trigger) {
     this.lateTrigger = trigger;
     return this;
   }
 
   @Override
-  public Window<M, K, WV, WindowPane<K, WV>> setAccumulationMode(AccumulationMode mode) {
+  public Window<M, K, WV> setAccumulationMode(AccumulationMode mode) {
     this.mode = mode;
     return this;
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/main/java/org/apache/samza/system/ExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/ExecutionEnvironment.java b/samza-api/src/main/java/org/apache/samza/system/ExecutionEnvironment.java
new file mode 100644
index 0000000..ad37eb3
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/ExecutionEnvironment.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.config.ConfigException;
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.apache.samza.config.Config;
+
+
+/**
+ * Interface to be implemented by physical execution engine to deploy the config and jobs to run the {@link org.apache.samza.operators.StreamGraph}
+ */
+@InterfaceStability.Unstable
+public interface ExecutionEnvironment {
+
+  String ENVIRONMENT_CONFIG = "job.execution.environment.class";
+  String DEFAULT_ENVIRONMENT_CLASS = "org.apache.samza.system.StandaloneExecutionEnvironment";
+
+  /**
+   * Static method to load the local standalone environment
+   *
+   * @param config  configuration passed in to initialize the Samza standalone process
+   * @return  the standalone {@link ExecutionEnvironment} to run the user-defined stream applications
+   */
+  static ExecutionEnvironment getLocalEnvironment(Config config) {
+    return null;
+  }
+
+  /**
+   * Static method to load the non-standalone environment.
+   *
+   * @param config  configuration passed in to initialize the Samza processes
+   * @return  the configure-driven {@link ExecutionEnvironment} to run the user-defined stream applications
+   */
+  static ExecutionEnvironment fromConfig(Config config) {
+    try {
+      if (ExecutionEnvironment.class.isAssignableFrom(Class.forName(config.get(ENVIRONMENT_CONFIG, DEFAULT_ENVIRONMENT_CLASS)))) {
+        return (ExecutionEnvironment) Class.forName(config.get(ENVIRONMENT_CONFIG, DEFAULT_ENVIRONMENT_CLASS)).newInstance();
+      }
+    } catch (Exception e) {
+      throw new ConfigException(String.format("Problem in loading ExecutionEnvironment class %s", config.get(ENVIRONMENT_CONFIG)), e);
+    }
+    throw new ConfigException(String.format(
+        "Class %s does not implement interface ExecutionEnvironment properly",
+        config.get(ENVIRONMENT_CONFIG)));
+  }
+
+  /**
+   * Method to be invoked to deploy and run the actual Samza jobs to execute {@link org.apache.samza.operators.StreamGraph}
+   *
+   * @param graphBuilder  the user-defined {@link StreamGraphBuilder} object
+   * @param config  the {@link Config} object for this job
+   */
+  void run(StreamGraphBuilder graphBuilder, Config config);
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/main/java/org/apache/samza/task/TaskContext.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/task/TaskContext.java b/samza-api/src/main/java/org/apache/samza/task/TaskContext.java
index a85e0b4..128cff1 100644
--- a/samza-api/src/main/java/org/apache/samza/task/TaskContext.java
+++ b/samza-api/src/main/java/org/apache/samza/task/TaskContext.java
@@ -54,4 +54,14 @@ public interface TaskContext {
    *
    */
   void setStartingOffset(SystemStreamPartition ssp, String offset);
+
+  /**
+   * Method to allow user to return customized context
+   *
+   * @param <T>  the type of user-defined task context
+   * @return  user-defined task context object
+   */
+  default <T> T getUserDefinedContext() {
+    return null;
+  };
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/test/java/org/apache/samza/operators/data/TestIncomingSystemMessage.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/data/TestIncomingSystemMessage.java b/samza-api/src/test/java/org/apache/samza/operators/data/TestIncomingSystemMessage.java
index 7bd62a7..e3a1290 100644
--- a/samza-api/src/test/java/org/apache/samza/operators/data/TestIncomingSystemMessage.java
+++ b/samza-api/src/test/java/org/apache/samza/operators/data/TestIncomingSystemMessage.java
@@ -33,7 +33,7 @@ public class TestIncomingSystemMessage {
   @Test
   public void testConstructor() {
     IncomingMessageEnvelope ime = mock(IncomingMessageEnvelope.class);
-    IncomingSystemMessageEnvelope ism = new IncomingSystemMessageEnvelope(ime);
+    InputMessageEnvelope ism = new InputMessageEnvelope(ime);
 
     Object mockKey = mock(Object.class);
     Object mockValue = mock(Object.class);

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowOutput.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowOutput.java b/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowOutput.java
deleted file mode 100644
index 9679e1d..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowOutput.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators.windows;
-
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-
-
-public class TestWindowOutput {
-  @Test
-  public void testConstructor() {
-    WindowPane<String, Integer> wndOutput = WindowPane.of(new WindowKey("testMsg", null), 10);
-    assertEquals(wndOutput.getKey().getKey(), "testMsg");
-    assertEquals(wndOutput.getMessage(), Integer.valueOf(10));
-    assertFalse(wndOutput.isDelete());
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/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
new file mode 100644
index 0000000..54d0b2f
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowPane.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.windows;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+
+public class TestWindowPane {
+  @Test
+  public void testConstructor() {
+    WindowPane<String, Integer> wndOutput = WindowPane.of(new WindowKey<>("testMsg", null), 10);
+    assertEquals(wndOutput.getKey().getKey(), "testMsg");
+    assertEquals(wndOutput.getMessage(), Integer.valueOf(10));
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
index 286893c..830e4a5 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
@@ -19,107 +19,156 @@
 
 package org.apache.samza.operators;
 
-import org.apache.samza.operators.data.MessageEnvelope;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.function.Function;
+import java.util.HashSet;
+import java.util.Set;
+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.JoinFunction;
 import org.apache.samza.operators.functions.MapFunction;
+import org.apache.samza.operators.functions.PartialJoinFunction;
 import org.apache.samza.operators.functions.SinkFunction;
 import org.apache.samza.operators.spec.OperatorSpec;
 import org.apache.samza.operators.spec.OperatorSpecs;
 import org.apache.samza.operators.windows.Window;
 import org.apache.samza.operators.windows.WindowPane;
 import org.apache.samza.operators.windows.internal.WindowInternal;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.function.BiFunction;
+import org.apache.samza.task.TaskContext;
 
 
 /**
  * The implementation for input/output {@link MessageStream}s to/from the operators.
  * Users use the {@link MessageStream} API methods to describe and chain the operators specs.
  *
- * @param <M>  type of {@link MessageEnvelope}s in this {@link MessageStream}
+ * @param <M>  type of messages in this {@link MessageStream}
  */
-public class MessageStreamImpl<M extends MessageEnvelope> implements MessageStream<M> {
+public class MessageStreamImpl<M> implements MessageStream<M> {
+  /**
+   * The {@link StreamGraphImpl} object that contains this {@link MessageStreamImpl}
+   */
+  private final StreamGraphImpl graph;
 
   /**
-   * The set of operators that consume the {@link MessageEnvelope}s in this {@link MessageStream}
+   * The set of operators that consume the messages in this {@link MessageStream}
    */
   private final Set<OperatorSpec> registeredOperatorSpecs = new HashSet<>();
 
-  @Override
-  public <OM extends MessageEnvelope> MessageStream<OM> map(MapFunction<M, OM> mapFn) {
-    OperatorSpec<OM> op = OperatorSpecs.<M, OM>createStreamOperatorSpec(m -> new ArrayList<OM>() { {
-        OM r = mapFn.apply(m);
-        if (r != null) {
-          this.add(r);
-        }
-      } });
+  /**
+   * Default constructor
+   *
+   * @param graph the {@link StreamGraphImpl} object that this stream belongs to
+   */
+  MessageStreamImpl(StreamGraphImpl graph) {
+    this.graph = graph;
+  }
+
+  @Override public <TM> MessageStream<TM> map(MapFunction<M, TM> mapFn) {
+    OperatorSpec<TM> op = OperatorSpecs.<M, TM>createMapOperatorSpec(mapFn, this.graph, new MessageStreamImpl<>(this.graph));
     this.registeredOperatorSpecs.add(op);
-    return op.getOutputStream();
+    return op.getNextStream();
   }
 
-  @Override
-  public <OM extends MessageEnvelope> MessageStream<OM> flatMap(FlatMapFunction<M, OM> flatMapFn) {
-    OperatorSpec<OM> op = OperatorSpecs.createStreamOperatorSpec(flatMapFn);
+  @Override public MessageStream<M> filter(FilterFunction<M> filterFn) {
+    OperatorSpec<M> op = OperatorSpecs.<M>createFilterOperatorSpec(filterFn, this.graph, new MessageStreamImpl<>(this.graph));
     this.registeredOperatorSpecs.add(op);
-    return op.getOutputStream();
+    return op.getNextStream();
   }
 
   @Override
-  public MessageStream<M> filter(FilterFunction<M> filterFn) {
-    OperatorSpec<M> op = OperatorSpecs.<M, M>createStreamOperatorSpec(t -> new ArrayList<M>() { {
-        if (filterFn.apply(t)) {
-          this.add(t);
-        }
-      } });
+  public <TM> MessageStream<TM> flatMap(FlatMapFunction<M, TM> flatMapFn) {
+    OperatorSpec<TM> op = OperatorSpecs.createStreamOperatorSpec(flatMapFn, this.graph, new MessageStreamImpl<>(this.graph));
     this.registeredOperatorSpecs.add(op);
-    return op.getOutputStream();
+    return op.getNextStream();
   }
 
   @Override
   public void sink(SinkFunction<M> sinkFn) {
-    this.registeredOperatorSpecs.add(OperatorSpecs.createSinkOperatorSpec(sinkFn));
+    this.registeredOperatorSpecs.add(OperatorSpecs.createSinkOperatorSpec(sinkFn, this.graph));
   }
 
-  @Override
-  public <K, WV, WM extends WindowPane<K, WV>> MessageStream<WM> window(
-      Window<M, K, WV, WM> window) {
-    OperatorSpec<WM> wndOp = OperatorSpecs.createWindowOperatorSpec((WindowInternal<MessageEnvelope, K, WV>) window);
-    this.registeredOperatorSpecs.add(wndOp);
-    return wndOp.getOutputStream();
+  @Override public void sendTo(OutputStream<M> stream) {
+    this.registeredOperatorSpecs.add(OperatorSpecs.createSendToOperatorSpec(stream.getSinkFunction(), this.graph, stream));
   }
 
   @Override
-  public <K, JM extends MessageEnvelope<K, ?>, RM extends MessageEnvelope> MessageStream<RM> join(
-      MessageStream<JM> otherStream, JoinFunction<M, JM, RM> joinFn) {
-    MessageStreamImpl<RM> outputStream = new MessageStreamImpl<>();
+  public <K, WV> MessageStream<WindowPane<K, WV>> window(Window<M, K, WV> window) {
+    OperatorSpec<WindowPane<K, WV>> wndOp = OperatorSpecs.createWindowOperatorSpec((WindowInternal<M, K, WV>) window,
+        this.graph, new MessageStreamImpl<>(this.graph));
+    this.registeredOperatorSpecs.add(wndOp);
+    return wndOp.getNextStream();
+  }
 
-    BiFunction<M, JM, RM> parJoin1 = joinFn::apply;
-    BiFunction<JM, M, RM> parJoin2 = (m, t1) -> joinFn.apply(t1, m);
+  @Override public <K, OM, RM> MessageStream<RM> join(MessageStream<OM> otherStream, JoinFunction<K, M, OM, RM> joinFn) {
+    MessageStreamImpl<RM> outputStream = new MessageStreamImpl<>(this.graph);
+
+    PartialJoinFunction<K, M, OM, RM> parJoin1 = new PartialJoinFunction<K, M, OM, RM>() {
+      @Override
+      public RM apply(M m1, OM om) {
+        return joinFn.apply(m1, om);
+      }
+
+      @Override
+      public K getKey(M message) {
+        return joinFn.getFirstKey(message);
+      }
+
+      @Override
+      public K getOtherKey(OM message) {
+        return joinFn.getSecondKey(message);
+      }
+
+      @Override
+      public void init(Config config, TaskContext context) {
+        joinFn.init(config, context);
+      }
+    };
+
+    PartialJoinFunction<K, OM, M, RM> parJoin2 = new PartialJoinFunction<K, OM, M, RM>() {
+      @Override
+      public RM apply(OM m1, M m) {
+        return joinFn.apply(m, m1);
+      }
+
+      @Override
+      public K getKey(OM message) {
+        return joinFn.getSecondKey(message);
+      }
+
+      @Override
+      public K getOtherKey(M message) {
+        return joinFn.getFirstKey(message);
+      }
+    };
 
     // TODO: need to add default store functions for the two partial join functions
 
-    ((MessageStreamImpl<JM>) otherStream).registeredOperatorSpecs.add(OperatorSpecs.createPartialJoinOperatorSpec(parJoin2, outputStream));
-    this.registeredOperatorSpecs.add(OperatorSpecs.createPartialJoinOperatorSpec(parJoin1, outputStream));
+    ((MessageStreamImpl<OM>) otherStream).registeredOperatorSpecs.add(
+        OperatorSpecs.<OM, K, M, RM>createPartialJoinOperatorSpec(parJoin2, this.graph, outputStream));
+    this.registeredOperatorSpecs.add(OperatorSpecs.<M, K, OM, RM>createPartialJoinOperatorSpec(parJoin1, this.graph, outputStream));
     return outputStream;
   }
 
   @Override
   public MessageStream<M> merge(Collection<MessageStream<M>> otherStreams) {
-    MessageStreamImpl<M> outputStream = new MessageStreamImpl<>();
+    MessageStreamImpl<M> outputStream = new MessageStreamImpl<>(this.graph);
 
     otherStreams.add(this);
-    otherStreams.forEach(other ->
-        ((MessageStreamImpl<M>) other).registeredOperatorSpecs.add(OperatorSpecs.createMergeOperatorSpec(outputStream)));
+    otherStreams.forEach(other -> ((MessageStreamImpl<M>) other).registeredOperatorSpecs.
+        add(OperatorSpecs.createMergeOperatorSpec(this.graph, outputStream)));
     return outputStream;
   }
 
+  @Override
+  public <K> MessageStream<M> partitionBy(Function<M, K> parKeyExtractor) {
+    MessageStreamImpl<M> intStream = this.graph.createIntStream(parKeyExtractor);
+    OutputStream<M> outputStream = this.graph.getOutputStream(intStream);
+    this.registeredOperatorSpecs.add(OperatorSpecs.createPartitionOperatorSpec(outputStream.getSinkFunction(),
+        this.graph, outputStream));
+    return intStream;
+  }
   /**
    * Gets the operator specs registered to consume the output of this {@link MessageStream}. This is an internal API and
    * should not be exposed to users.
@@ -129,4 +178,5 @@ public class MessageStreamImpl<M extends MessageEnvelope> implements MessageStre
   public Collection<OperatorSpec> getRegisteredOperatorSpecs() {
     return Collections.unmodifiableSet(this.registeredOperatorSpecs);
   }
+
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/StreamGraphImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
new file mode 100644
index 0000000..dca3469
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
@@ -0,0 +1,260 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators;
+
+import java.util.Properties;
+import java.util.function.Function;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * The implementation of {@link StreamGraph} interface. This class provides implementation of methods to allow users to
+ * create system input/output/intermediate streams.
+ */
+public class StreamGraphImpl implements StreamGraph {
+
+  /**
+   * Unique identifier for each {@link org.apache.samza.operators.spec.OperatorSpec} added to transform the {@link MessageEnvelope}
+   * in the input {@link MessageStream}s.
+   */
+  private int opId = 0;
+
+  private class InputStreamImpl<K, V, M extends MessageEnvelope<K, V>> extends MessageStreamImpl<M> {
+    final StreamSpec spec;
+    final Serde<K> keySerde;
+    final Serde<V> msgSerde;
+
+    InputStreamImpl(StreamGraphImpl graph, StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
+      super(graph);
+      this.spec = streamSpec;
+      this.keySerde = keySerde;
+      this.msgSerde = msgSerde;
+    }
+
+    StreamSpec getSpec() {
+      return this.spec;
+    }
+
+  }
+
+  private class OutputStreamImpl<K, V, M extends MessageEnvelope<K, V>> implements OutputStream<M> {
+    final StreamSpec spec;
+    final Serde<K> keySerde;
+    final Serde<V> msgSerde;
+
+    OutputStreamImpl(StreamGraphImpl graph, StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
+      this.spec = streamSpec;
+      this.keySerde = keySerde;
+      this.msgSerde = msgSerde;
+    }
+
+    StreamSpec getSpec() {
+      return this.spec;
+    }
+
+    @Override
+    public SinkFunction<M> getSinkFunction() {
+      return (M message, MessageCollector mc, TaskCoordinator tc) -> {
+        // TODO: need to find a way to directly pass in the serde class names
+        // mc.send(new OutgoingMessageEnvelope(this.spec.getSystemStream(), this.keySerde.getClass().getName(), this.msgSerde.getClass().getName(),
+        //    message.getKey(), message.getKey(), message.getMessage()));
+        mc.send(new OutgoingMessageEnvelope(this.spec.getSystemStream(), message.getKey(), message.getMessage()));
+      };
+    }
+  }
+
+  private class IntermediateStreamImpl<PK, K, V, M extends MessageEnvelope<K, V>> extends InputStreamImpl<K, V, M> implements OutputStream<M> {
+    final Function<M, PK> parKeyFn;
+
+    /**
+     * Default constructor
+     *
+     * @param graph the {@link StreamGraphImpl} object that this stream belongs to
+     */
+    IntermediateStreamImpl(StreamGraphImpl graph, StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
+      this(graph, streamSpec, keySerde, msgSerde, null);
+    }
+
+    IntermediateStreamImpl(StreamGraphImpl graph, StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde, Function<M, PK> parKeyFn) {
+      super(graph, streamSpec, keySerde, msgSerde);
+      this.parKeyFn = parKeyFn;
+    }
+
+    @Override
+    public SinkFunction<M> getSinkFunction() {
+      return (M message, MessageCollector mc, TaskCoordinator tc) -> {
+        // TODO: need to find a way to directly pass in the serde class names
+        // mc.send(new OutgoingMessageEnvelope(this.spec.getSystemStream(), this.keySerde.getClass().getName(), this.msgSerde.getClass().getName(),
+        //    message.getKey(), message.getKey(), message.getMessage()));
+        if (this.parKeyFn == null) {
+          mc.send(new OutgoingMessageEnvelope(this.spec.getSystemStream(), message.getKey(), message.getMessage()));
+        } else {
+          // apply partition key function
+          mc.send(new OutgoingMessageEnvelope(this.spec.getSystemStream(), this.parKeyFn.apply(message), message.getKey(), message.getMessage()));
+        }
+      };
+    }
+  }
+
+  /**
+   * Maps keeping all {@link SystemStream}s that are input and output of operators in {@link StreamGraphImpl}
+   */
+  private final Map<SystemStream, MessageStream> inStreams = new HashMap<>();
+  private final Map<SystemStream, OutputStream> outStreams = new HashMap<>();
+
+  private ContextManager contextManager = new ContextManager() { };
+
+  @Override
+  public <K, V, M extends MessageEnvelope<K, V>> MessageStream<M> createInStream(StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
+    if (!this.inStreams.containsKey(streamSpec.getSystemStream())) {
+      this.inStreams.putIfAbsent(streamSpec.getSystemStream(), new InputStreamImpl<K, V, M>(this, streamSpec, keySerde, msgSerde));
+    }
+    return this.inStreams.get(streamSpec.getSystemStream());
+  }
+
+  /**
+   * Helper method to be used by {@link MessageStreamImpl} class
+   *
+   * @param streamSpec  the {@link StreamSpec} object defining the {@link SystemStream} as the output
+   * @param <M>  the type of {@link MessageEnvelope}s in the output {@link SystemStream}
+   * @return  the {@link MessageStreamImpl} object
+   */
+  @Override
+  public <K, V, M extends MessageEnvelope<K, V>> OutputStream<M> createOutStream(StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
+    if (!this.outStreams.containsKey(streamSpec.getSystemStream())) {
+      this.outStreams.putIfAbsent(streamSpec.getSystemStream(), new OutputStreamImpl<K, V, M>(this, streamSpec, keySerde, msgSerde));
+    }
+    return this.outStreams.get(streamSpec.getSystemStream());
+  }
+
+  /**
+   * Helper method to be used by {@link MessageStreamImpl} class
+   *
+   * @param streamSpec  the {@link StreamSpec} object defining the {@link SystemStream} as an intermediate {@link SystemStream}
+   * @param <M>  the type of {@link MessageEnvelope}s in the output {@link SystemStream}
+   * @return  the {@link MessageStreamImpl} object
+   */
+  @Override
+  public <K, V, M extends MessageEnvelope<K, V>> OutputStream<M> createIntStream(StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
+    if (!this.inStreams.containsKey(streamSpec.getSystemStream())) {
+      this.inStreams.putIfAbsent(streamSpec.getSystemStream(), new IntermediateStreamImpl<K, K, V, M>(this, streamSpec, keySerde, msgSerde));
+    }
+    IntermediateStreamImpl<K, K, V, M> intStream = (IntermediateStreamImpl<K, K, V, M>) this.inStreams.get(streamSpec.getSystemStream());
+    if (!this.outStreams.containsKey(streamSpec.getSystemStream())) {
+      this.outStreams.putIfAbsent(streamSpec.getSystemStream(), intStream);
+    }
+    return intStream;
+  }
+
+  @Override public Map<StreamSpec, MessageStream> getInStreams() {
+    Map<StreamSpec, MessageStream> inStreamMap = new HashMap<>();
+    this.inStreams.forEach((ss, entry) -> inStreamMap.put(((InputStreamImpl) entry).getSpec(), entry));
+    return Collections.unmodifiableMap(inStreamMap);
+  }
+
+  @Override public Map<StreamSpec, OutputStream> getOutStreams() {
+    Map<StreamSpec, OutputStream> outStreamMap = new HashMap<>();
+    this.outStreams.forEach((ss, entry) -> outStreamMap.put(((OutputStreamImpl) entry).getSpec(), entry));
+    return Collections.unmodifiableMap(outStreamMap);
+  }
+
+  @Override
+  public StreamGraph withContextManager(ContextManager manager) {
+    this.contextManager = manager;
+    return this;
+  }
+
+  public int getNextOpId() {
+    return this.opId++;
+  }
+
+  public ContextManager getContextManager() {
+    return this.contextManager;
+  }
+
+  /**
+   * Helper method to be get the input stream via {@link SystemStream}
+   *
+   * @param systemStream  the {@link SystemStream}
+   * @return  a {@link MessageStreamImpl} object corresponding to the {@code systemStream}
+   */
+  public MessageStreamImpl getInputStream(SystemStream systemStream) {
+    if (this.inStreams.containsKey(systemStream)) {
+      return (MessageStreamImpl) this.inStreams.get(systemStream);
+    }
+    return null;
+  }
+
+  <M> OutputStream<M> getOutputStream(MessageStreamImpl<M> intStream) {
+    if (this.outStreams.containsValue(intStream)) {
+      return (OutputStream<M>) intStream;
+    }
+    return null;
+  }
+
+  <M> MessageStream<M> getIntStream(OutputStream<M> outStream) {
+    if (this.inStreams.containsValue(outStream)) {
+      return (MessageStream<M>) outStream;
+    }
+    return null;
+  }
+
+  /**
+   * Method to create intermediate topics for {@link MessageStreamImpl#partitionBy(Function)} method.
+   *
+   * @param parKeyFn  the function to extract the partition key from the input message
+   * @param <PK>  the type of partition key
+   * @param <M>  the type of input message
+   * @return  the {@link OutputStream} object for the re-partitioned stream
+   */
+  <PK, M> MessageStreamImpl<M> createIntStream(Function<M, PK> parKeyFn) {
+    // TODO: placeholder to auto-generate intermediate streams via {@link StreamSpec}
+    StreamSpec streamSpec = new StreamSpec() {
+      @Override
+      public SystemStream getSystemStream() {
+        // TODO: should auto-generate intermedaite stream name here
+        return new SystemStream("intermediate", String.format("par-%d", StreamGraphImpl.this.opId));
+      }
+
+      @Override
+      public Properties getProperties() {
+        return null;
+      }
+    };
+
+    if (!this.inStreams.containsKey(streamSpec.getSystemStream())) {
+      this.inStreams.putIfAbsent(streamSpec.getSystemStream(), new IntermediateStreamImpl(this, streamSpec, null, null, parKeyFn));
+    }
+    IntermediateStreamImpl intStream = (IntermediateStreamImpl) this.inStreams.get(streamSpec.getSystemStream());
+    if (!this.outStreams.containsKey(streamSpec.getSystemStream())) {
+      this.outStreams.putIfAbsent(streamSpec.getSystemStream(), intStream);
+    }
+    return intStream;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/operators/StreamOperatorAdaptorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/StreamOperatorAdaptorTask.java b/samza-operator/src/main/java/org/apache/samza/operators/StreamOperatorAdaptorTask.java
deleted file mode 100644
index 152cd92..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/StreamOperatorAdaptorTask.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.data.IncomingSystemMessageEnvelope;
-import org.apache.samza.operators.data.MessageEnvelope;
-import org.apache.samza.operators.impl.OperatorImpl;
-import org.apache.samza.operators.impl.OperatorImpls;
-import org.apache.samza.system.IncomingMessageEnvelope;
-import org.apache.samza.system.SystemStreamPartition;
-import org.apache.samza.task.InitableTask;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.StreamTask;
-import org.apache.samza.task.TaskContext;
-import org.apache.samza.task.TaskCoordinator;
-import org.apache.samza.task.WindowableTask;
-
-import java.util.HashMap;
-import java.util.Map;
-
-
-/**
- * An {@link StreamTask} implementation that receives {@link IncomingSystemMessageEnvelope}s and propagates them
- * through the user's stream transformations defined in {@link StreamOperatorTask#transform(Map)} using the
- * {@link MessageStream} APIs.
- * <p>
- * This class brings all the operator API implementation components together and feeds the
- * {@link IncomingSystemMessageEnvelope}s into the transformation chains.
- * <p>
- * It accepts an instance of the user implemented {@link StreamOperatorTask}. When its own {@link #init(Config, TaskContext)}
- * method is called during startup, it creates a {@link MessageStreamImpl} corresponding to each of its input
- * {@link SystemStreamPartition}s and then calls the user's {@link StreamOperatorTask#transform(Map)} method.
- * <p>
- * When users invoke the methods on the {@link MessageStream} API to describe their stream transformations in the
- * {@link StreamOperatorTask#transform(Map)} method, the underlying {@link MessageStreamImpl} creates the
- * corresponding {@link org.apache.samza.operators.spec.OperatorSpec} to record information about the desired
- * transformation, and returns the output {@link MessageStream} to allow further transform chaining.
- * <p>
- * Once the user's transformation DAGs have been described for all {@link MessageStream}s (i.e., when the
- * {@link StreamOperatorTask#transform(Map)} call returns), it calls
- * {@link OperatorImpls#createOperatorImpls(MessageStreamImpl, TaskContext)} for each of the input
- * {@link MessageStreamImpl}. This instantiates the {@link org.apache.samza.operators.impl.OperatorImpl} DAG
- * corresponding to the aforementioned {@link org.apache.samza.operators.spec.OperatorSpec} DAG and returns the
- * root node of the DAG, which this class saves.
- * <p>
- * Now that it has the root for the DAG corresponding to each {@link SystemStreamPartition}, it can pass the message
- * envelopes received in {@link StreamTask#process(IncomingMessageEnvelope, MessageCollector, TaskCoordinator)} along
- * to the appropriate root nodes. From then on, each {@link org.apache.samza.operators.impl.OperatorImpl} propagates
- * its transformed output to the next set of {@link org.apache.samza.operators.impl.OperatorImpl}s.
- */
-public final class StreamOperatorAdaptorTask implements StreamTask, InitableTask, WindowableTask {
-
-  /**
-   * A mapping from each {@link SystemStreamPartition} to the root node of its operator chain DAG.
-   */
-  private final Map<SystemStreamPartition, OperatorImpl<IncomingSystemMessageEnvelope, ? extends MessageEnvelope>> operatorChains = new HashMap<>();
-
-  private final StreamOperatorTask userTask;
-
-  public StreamOperatorAdaptorTask(StreamOperatorTask userTask) {
-    this.userTask = userTask;
-  }
-
-  @Override
-  public final void init(Config config, TaskContext context) throws Exception {
-    if (this.userTask instanceof InitableTask) {
-      ((InitableTask) this.userTask).init(config, context);
-    }
-    Map<SystemStreamPartition, MessageStream<IncomingSystemMessageEnvelope>> messageStreams = new HashMap<>();
-    context.getSystemStreamPartitions().forEach(ssp -> messageStreams.put(ssp, new MessageStreamImpl<>()));
-    this.userTask.transform(messageStreams);
-    messageStreams.forEach((ssp, ms) ->
-        operatorChains.put(ssp, OperatorImpls.createOperatorImpls((MessageStreamImpl<IncomingSystemMessageEnvelope>) ms, context)));
-  }
-
-  @Override
-  public final void process(IncomingMessageEnvelope ime, MessageCollector collector, TaskCoordinator coordinator) {
-    this.operatorChains.get(ime.getSystemStreamPartition())
-        .onNext(new IncomingSystemMessageEnvelope(ime), collector, coordinator);
-  }
-
-  @Override
-  public final void window(MessageCollector collector, TaskCoordinator coordinator) throws Exception {
-    if (this.userTask instanceof WindowableTask) {
-      ((WindowableTask) this.userTask).window(collector, coordinator);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java b/samza-operator/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java
new file mode 100644
index 0000000..809a70a
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.functions;
+
+import org.apache.samza.annotation.InterfaceStability;
+
+
+/**
+ * This defines the interface function a two-way join functions that takes input messages from two input
+ * {@link org.apache.samza.operators.MessageStream}s and merge them into a single output joined message in the join output
+ */
+@InterfaceStability.Unstable
+public interface PartialJoinFunction<K, M, OM, RM> extends InitableFunction {
+
+  /**
+   * Method to perform join method on the two input messages
+   *
+   * @param m1  message from the first input stream
+   * @param om  message from the second input stream
+   * @return  the joined message in the output stream
+   */
+  RM apply(M m1, OM om);
+
+  /**
+   * Method to get the key from the input message
+   *
+   * @param message  the input message from the first strean
+   * @return  the join key in the {@code message}
+   */
+  K getKey(M message);
+
+  /**
+   * Method to get the key from the input message in the other stream
+   *
+   * @param message  the input message from the other stream
+   * @return  the join key in the {@code message}
+   */
+  K getOtherKey(OM message);
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorGraph.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorGraph.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorGraph.java
new file mode 100644
index 0000000..66336f8
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorGraph.java
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.impl;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.spec.OperatorSpec;
+import org.apache.samza.operators.spec.PartialJoinOperatorSpec;
+import org.apache.samza.operators.spec.SinkOperatorSpec;
+import org.apache.samza.operators.spec.StreamOperatorSpec;
+import org.apache.samza.operators.spec.WindowOperatorSpec;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.task.TaskContext;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+
+/**
+ * Instantiates the DAG of {@link OperatorImpl}s corresponding to the {@link OperatorSpec}s for a
+ * {@link MessageStreamImpl}
+ */
+public class OperatorGraph {
+
+  /**
+   * A {@link Map} from {@link OperatorSpec} to {@link OperatorImpl}. This map registers all {@link OperatorImpl} in the DAG
+   * of {@link OperatorImpl} in a {@link org.apache.samza.container.TaskInstance}. Each {@link OperatorImpl} is created
+   * according to a single instance of {@link OperatorSpec}.
+   */
+  private final Map<OperatorSpec, OperatorImpl> operators = new HashMap<>();
+
+  /**
+   * This {@link Map} describes the DAG of {@link OperatorImpl} that are chained together to process the input messages.
+   */
+  private final Map<SystemStream, RootOperatorImpl> operatorGraph = new HashMap<>();
+
+  /**
+   * Initialize the whole DAG of {@link OperatorImpl}s, based on the input {@link MessageStreamImpl} from the {@link org.apache.samza.operators.StreamGraph}.
+   * This method will traverse each input {@link org.apache.samza.operators.MessageStream} in the {@code inputStreams} and
+   * instantiate the corresponding {@link OperatorImpl} chains that take the {@link org.apache.samza.operators.MessageStream} as input.
+   *
+   * @param inputStreams  the map of input {@link org.apache.samza.operators.MessageStream}s
+   * @param config  the {@link Config} required to instantiate operators
+   * @param context  the {@link TaskContext} required to instantiate operators
+   */
+  public void init(Map<SystemStream, MessageStreamImpl> inputStreams, Config config, TaskContext context) {
+    inputStreams.forEach((ss, mstream) -> this.operatorGraph.put(ss, this.createOperatorImpls(mstream, config, context)));
+  }
+
+  /**
+   * Method to get the corresponding {@link RootOperatorImpl}
+   *
+   * @param ss  input {@link SystemStream}
+   * @param <M>  the type of input message
+   * @return  the {@link OperatorImpl} that starts processing the input message
+   */
+  public <M> OperatorImpl<M, M> get(SystemStream ss) {
+    return this.operatorGraph.get(ss);
+  }
+
+  /**
+   * Traverses the DAG of {@link OperatorSpec}s starting from the provided {@link MessageStreamImpl},
+   * creates the corresponding DAG of {@link OperatorImpl}s, and returns its root {@link RootOperatorImpl} node.
+   *
+   * @param source  the input {@link MessageStreamImpl} to instantiate {@link OperatorImpl}s for
+   * @param <M>  the type of messagess in the {@code source} {@link MessageStreamImpl}
+   * @param config  the {@link Config} required to instantiate operators
+   * @param context  the {@link TaskContext} required to instantiate operators
+   * @return  root node for the {@link OperatorImpl} DAG
+   */
+  private <M> RootOperatorImpl<M> createOperatorImpls(MessageStreamImpl<M> source, Config config,
+      TaskContext context) {
+    // since the source message stream might have multiple operator specs registered on it,
+    // create a new root node as a single point of entry for the DAG.
+    RootOperatorImpl<M> rootOperator = new RootOperatorImpl<>();
+    // create the pipeline/topology starting from the source
+    source.getRegisteredOperatorSpecs().forEach(registeredOperator -> {
+        // pass in the source and context s.t. stateful stream operators can initialize their stores
+        OperatorImpl<M, ?> operatorImpl =
+            this.createAndRegisterOperatorImpl(registeredOperator, source, config, context);
+        rootOperator.registerNextOperator(operatorImpl);
+      });
+    return rootOperator;
+  }
+
+  /**
+   * Helper method to recursively traverse the {@link OperatorSpec} DAG and instantiate and link the corresponding
+   * {@link OperatorImpl}s.
+   *
+   * @param operatorSpec  the operatorSpec registered with the {@code source}
+   * @param source  the source {@link MessageStreamImpl}
+   * @param <M>  type of input message
+   * @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
+   */
+  private <M> OperatorImpl<M, ?> createAndRegisterOperatorImpl(OperatorSpec operatorSpec,
+      MessageStreamImpl<M> source, Config config, TaskContext context) {
+    if (!operators.containsKey(operatorSpec)) {
+      OperatorImpl<M, ?> operatorImpl = createOperatorImpl(source, operatorSpec, config, context);
+      if (operators.putIfAbsent(operatorSpec, operatorImpl) == null) {
+        // this is the first time we've added the operatorImpl corresponding to the operatorSpec,
+        // so traverse and initialize and register the rest of the DAG.
+        // initialize the corresponding operator function
+        operatorSpec.init(config, context);
+        MessageStreamImpl nextStream = operatorSpec.getNextStream();
+        if (nextStream != null) {
+          Collection<OperatorSpec> registeredSpecs = nextStream.getRegisteredOperatorSpecs();
+          registeredSpecs.forEach(registeredSpec -> {
+              OperatorImpl subImpl = this.createAndRegisterOperatorImpl(registeredSpec, nextStream, config, context);
+              operatorImpl.registerNextOperator(subImpl);
+            });
+        }
+        return operatorImpl;
+      }
+    }
+
+    // the implementation corresponding to operatorSpec has already been instantiated
+    // and registered, so we do not need to traverse the DAG further.
+    return operators.get(operatorSpec);
+  }
+
+  /**
+   * Creates a new {@link OperatorImpl} instance for the provided {@link OperatorSpec}.
+   *
+   * @param source  the source {@link MessageStreamImpl}
+   * @param <M>  type of input message
+   * @param operatorSpec  the immutable {@link OperatorSpec} definition.
+   * @param config  the {@link Config} required to instantiate operators
+   * @param context  the {@link TaskContext} required to instantiate operators
+   * @return  the {@link OperatorImpl} implementation instance
+   */
+  private static <M> OperatorImpl<M, ?> createOperatorImpl(MessageStreamImpl<M> source, OperatorSpec operatorSpec, Config config, TaskContext context) {
+    if (operatorSpec instanceof StreamOperatorSpec) {
+      StreamOperatorSpec<M, ?> streamOpSpec = (StreamOperatorSpec<M, ?>) operatorSpec;
+      return new StreamOperatorImpl<>(streamOpSpec, source, config, context);
+    } else if (operatorSpec instanceof SinkOperatorSpec) {
+      return new SinkOperatorImpl<>((SinkOperatorSpec<M>) operatorSpec, config, context);
+    } else if (operatorSpec instanceof WindowOperatorSpec) {
+      return new WindowOperatorImpl<>((WindowOperatorSpec<M, ?, ?>) operatorSpec, source, config, context);
+    } else if (operatorSpec instanceof PartialJoinOperatorSpec) {
+      return new PartialJoinOperatorImpl<>((PartialJoinOperatorSpec) operatorSpec, source, config, context);
+    }
+    throw new IllegalArgumentException(
+        String.format("Unsupported OperatorSpec: %s", operatorSpec.getClass().getName()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
index c77914e..abb1fa9 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
@@ -18,10 +18,7 @@
  */
 package org.apache.samza.operators.impl;
 
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
 
 import java.util.HashSet;
@@ -31,32 +28,24 @@ import java.util.Set;
 /**
  * Abstract base class for all stream operator implementations.
  */
-public abstract class OperatorImpl<M extends MessageEnvelope, RM extends MessageEnvelope> {
+public abstract class OperatorImpl<M, RM> {
 
-  private final Set<OperatorImpl<RM, ? extends MessageEnvelope>> nextOperators = new HashSet<>();
+  private final Set<OperatorImpl<RM, ?>> nextOperators = new HashSet<>();
 
   /**
    * Register the next operator in the chain that this operator should propagate its output to.
    * @param nextOperator  the next operator in the chain.
    */
-  void registerNextOperator(OperatorImpl<RM, ? extends MessageEnvelope> nextOperator) {
+  void registerNextOperator(OperatorImpl<RM, ?> nextOperator) {
     nextOperators.add(nextOperator);
   }
 
   /**
-   * Initialize the initial state for stateful operators.
-   *
-   * @param source  the source that this {@link OperatorImpl} operator is registered with
-   * @param context  the task context to initialize the operator implementation
-   */
-  public void init(MessageStream<M> source, TaskContext context) {}
-
-  /**
    * Perform the transformation required for this operator and call the downstream operators.
    *
    * Must call {@link #propagateResult} to propage the output to registered downstream operators correctly.
    *
-   * @param message  the input {@link MessageEnvelope}
+   * @param message  the input message
    * @param collector  the {@link MessageCollector} in the context
    * @param coordinator  the {@link TaskCoordinator} in the context
    */
@@ -67,11 +56,12 @@ public abstract class OperatorImpl<M extends MessageEnvelope, RM extends Message
    *
    * This method <b>must</b> be called from {@link #onNext} to propagate the operator output correctly.
    *
-   * @param outputMessage  output {@link MessageEnvelope}
+   * @param outputMessage  output message
    * @param collector  the {@link MessageCollector} in the context
    * @param coordinator  the {@link TaskCoordinator} in the context
    */
   void propagateResult(RM outputMessage, MessageCollector collector, TaskCoordinator coordinator) {
     nextOperators.forEach(sub -> sub.onNext(outputMessage, collector, coordinator));
   }
+
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java
deleted file mode 100644
index 02095cb..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators.impl;
-
-
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.MessageStreamImpl;
-import org.apache.samza.operators.data.MessageEnvelope;
-import org.apache.samza.operators.spec.OperatorSpec;
-import org.apache.samza.operators.spec.PartialJoinOperatorSpec;
-import org.apache.samza.operators.spec.SinkOperatorSpec;
-import org.apache.samza.operators.spec.StreamOperatorSpec;
-import org.apache.samza.operators.spec.WindowOperatorSpec;
-import org.apache.samza.operators.windows.WindowPane;
-import org.apache.samza.task.TaskContext;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-
-/**
- * Instantiates the DAG of {@link OperatorImpl}s corresponding to the {@link OperatorSpec}s for a
- * {@link MessageStreamImpl}
- */
-public class OperatorImpls {
-
-  /**
-   * Holds the mapping between the {@link OperatorSpec} and {@link OperatorImpl}s instances.
-   */
-  private static final Map<OperatorSpec, OperatorImpl> OPERATOR_IMPLS = new ConcurrentHashMap<>();
-
-  /**
-   * Traverses the DAG of {@link OperatorSpec}s starting from the provided {@link MessageStreamImpl},
-   * creates the corresponding DAG of {@link OperatorImpl}s, and returns its root {@link RootOperatorImpl} node.
-   *
-   * @param source  the input {@link MessageStreamImpl} to instantiate {@link OperatorImpl}s for
-   * @param <M>  the type of {@link MessageEnvelope}s in the {@code source} {@link MessageStream}
-   * @param context  the {@link TaskContext} required to instantiate operators
-   * @return  root node for the {@link OperatorImpl} DAG
-   */
-  public static <M extends MessageEnvelope> RootOperatorImpl createOperatorImpls(MessageStreamImpl<M> source, TaskContext context) {
-    // since the source message stream might have multiple operator specs registered on it,
-    // create a new root node as a single point of entry for the DAG.
-    RootOperatorImpl<M> rootOperator = new RootOperatorImpl<>();
-    // create the pipeline/topology starting from the source
-    source.getRegisteredOperatorSpecs().forEach(registeredOperator -> {
-        // pass in the source and context s.t. stateful stream operators can initialize their stores
-        OperatorImpl<M, ? extends MessageEnvelope> operatorImpl =
-            createAndRegisterOperatorImpl(registeredOperator, source, context);
-        rootOperator.registerNextOperator(operatorImpl);
-      });
-    return rootOperator;
-  }
-
-  /**
-   * Helper method to recursively traverse the {@link OperatorSpec} DAG and instantiate and link the corresponding
-   * {@link OperatorImpl}s.
-   *
-   * @param operatorSpec  the operatorSpec registered with the {@code source}
-   * @param source  the source {@link MessageStreamImpl}
-   * @param context  the context of the task
-   * @return  the operator implementation for the operatorSpec
-   */
-  private static <M extends MessageEnvelope> OperatorImpl<M, ? extends MessageEnvelope> createAndRegisterOperatorImpl(OperatorSpec operatorSpec,
-      MessageStream source, TaskContext context) {
-    if (!OPERATOR_IMPLS.containsKey(operatorSpec)) {
-      OperatorImpl<M, ? extends MessageEnvelope> operatorImpl = createOperatorImpl(operatorSpec);
-      if (OPERATOR_IMPLS.putIfAbsent(operatorSpec, operatorImpl) == null) {
-        // this is the first time we've added the operatorImpl corresponding to the operatorSpec,
-        // so traverse and initialize and register the rest of the DAG.
-        MessageStream<? extends MessageEnvelope> outStream = operatorSpec.getOutputStream();
-        Collection<OperatorSpec> registeredSpecs = ((MessageStreamImpl) outStream).getRegisteredOperatorSpecs();
-        registeredSpecs.forEach(registeredSpec -> {
-            OperatorImpl subImpl = createAndRegisterOperatorImpl(registeredSpec, outStream, context);
-            operatorImpl.registerNextOperator(subImpl);
-          });
-        operatorImpl.init(source, context);
-        return operatorImpl;
-      }
-    }
-
-    // the implementation corresponding to operatorSpec has already been instantiated
-    // and registered, so we do not need to traverse the DAG further.
-    return OPERATOR_IMPLS.get(operatorSpec);
-  }
-
-  /**
-   * Creates a new {@link OperatorImpl} instance for the provided {@link OperatorSpec}.
-   *
-   * @param operatorSpec  the immutable {@link OperatorSpec} definition.
-   * @param <M>  type of input {@link MessageEnvelope}
-   * @return  the {@link OperatorImpl} implementation instance
-   */
-  protected static <M extends MessageEnvelope> OperatorImpl<M, ? extends MessageEnvelope> createOperatorImpl(OperatorSpec operatorSpec) {
-    if (operatorSpec instanceof StreamOperatorSpec) {
-      return new StreamOperatorImpl<>((StreamOperatorSpec<M, ? extends MessageEnvelope>) operatorSpec);
-    } else if (operatorSpec instanceof SinkOperatorSpec) {
-      return new SinkOperatorImpl<>((SinkOperatorSpec<M>) operatorSpec);
-    } else if (operatorSpec instanceof WindowOperatorSpec) {
-      return new WindowOperatorImpl<>((WindowOperatorSpec<M, ?, ?, ?, ? extends WindowPane>) operatorSpec);
-    } else if (operatorSpec instanceof PartialJoinOperatorSpec) {
-      return new PartialJoinOperatorImpl<>((PartialJoinOperatorSpec) operatorSpec);
-    }
-    throw new IllegalArgumentException(
-        String.format("Unsupported OperatorSpec: %s", operatorSpec.getClass().getName()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java
index 90569b4..c8515e1 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java
@@ -18,9 +18,11 @@
  */
 package org.apache.samza.operators.impl;
 
-import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
 import org.apache.samza.operators.spec.PartialJoinOperatorSpec;
 import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
 
 
@@ -28,14 +30,13 @@ import org.apache.samza.task.TaskCoordinator;
  * Implementation of a {@link PartialJoinOperatorSpec}. This class implements function
  * that only takes in one input stream among all inputs to the join and generate the join output.
  *
- * @param <M>  type of {@link MessageEnvelope}s in the input stream
- * @param <JM>  type of {@link MessageEnvelope}s in the stream to join with
- * @param <RM>  type of {@link MessageEnvelope}s in the joined stream
+ * @param <M>  type of messages in the input stream
+ * @param <JM>  type of messages in the stream to join with
+ * @param <RM>  type of messages in the joined stream
  */
-class PartialJoinOperatorImpl<M extends MessageEnvelope<K, ?>, K, JM extends MessageEnvelope<K, ?>, RM extends MessageEnvelope>
-    extends OperatorImpl<M, RM> {
+class PartialJoinOperatorImpl<M, K, JM, RM> extends OperatorImpl<M, RM> {
 
-  PartialJoinOperatorImpl(PartialJoinOperatorSpec<M, K, JM, RM> joinOp) {
+  PartialJoinOperatorImpl(PartialJoinOperatorSpec<M, K, JM, RM> joinOp, MessageStreamImpl<M> source, Config config, TaskContext context) {
     // TODO: implement PartialJoinOperatorImpl constructor
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.java
index 7132b86..4b30a5d 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.java
@@ -18,16 +18,15 @@
  */
 package org.apache.samza.operators.impl;
 
-import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.task.MessageCollector;
 import org.apache.samza.task.TaskCoordinator;
 
 
 /**
- * A no-op operator implementation that forwards incoming {@link MessageEnvelope}s to all of its subscribers.
- * @param <M>  type of incoming {@link MessageEnvelope}s
+ * A no-op operator implementation that forwards incoming messages to all of its subscribers.
+ * @param <M>  type of incoming messages
  */
-final class RootOperatorImpl<M extends MessageEnvelope> extends OperatorImpl<M, M> {
+final class RootOperatorImpl<M> extends OperatorImpl<M, M> {
 
   @Override
   public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java
new file mode 100644
index 0000000..2bb362c
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.impl;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.spec.WindowOperatorSpec;
+import org.apache.samza.operators.windows.WindowPane;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+
+
+/**
+ * Default implementation class of a {@link WindowOperatorSpec} for a session window.
+ *
+ * @param <M>  the type of input message
+ * @param <RK>  the type of window key
+ * @param <WV>  the type of window state
+ */
+class SessionWindowOperatorImpl<M, RK, WV> extends OperatorImpl<M, WindowPane<RK, WV>> {
+
+  private final WindowOperatorSpec<M, RK, WV> windowSpec;
+
+  SessionWindowOperatorImpl(WindowOperatorSpec<M, RK, WV> windowSpec, MessageStreamImpl<M> source, Config config, TaskContext context) {
+    this.windowSpec = windowSpec;
+  }
+
+  @Override
+  public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
+  }
+
+  public void onTimer(MessageCollector collector, TaskCoordinator coordinator) {
+    // This is to periodically check the timeout triggers to get the list of window states to be updated
+  }
+}


[06/14] samza git commit: SAMZA-1073: top-level fluent API `

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/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 f7e1f36..73fb5c8 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
@@ -20,7 +20,6 @@
 package org.apache.samza.operators.windows;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.operators.triggers.TimeTrigger;
 import org.apache.samza.operators.triggers.Trigger;
 import org.apache.samza.operators.triggers.Triggers;
@@ -34,11 +33,11 @@ import java.util.function.Function;
 /**
  * APIs for creating different types of {@link Window}s.
  *
- * Groups the incoming {@link MessageEnvelope}s in the {@link org.apache.samza.operators.MessageStream} into finite windows for processing.
+ * Groups the incoming messages in the {@link org.apache.samza.operators.MessageStream} into finite windows for processing.
  *
  * <p> Each window is uniquely identified by its {@link WindowKey}. A window can have one or more associated {@link Trigger}s
  * that determine when results from the {@link Window} are emitted. Each emitted result contains one or more
- * {@link MessageEnvelope}s in the window and is called a {@link WindowPane}.
+ * messages in the window and is called a {@link WindowPane}.
  *
  * <p> A window can have early triggers that allow emitting {@link WindowPane}s speculatively before all data for the window
  * has arrived or late triggers that allow handling of late data arrivals.
@@ -74,14 +73,14 @@ import java.util.function.Function;
  *   <li>
  *     Session Windows: A session window groups a {@link org.apache.samza.operators.MessageStream} into sessions.
  *     A <i>session</i> captures some period of activity over a {@link org.apache.samza.operators.MessageStream}.
- *     The boundary for a session is defined by a {@code sessionGap}. All {@link MessageEnvelope}s that that arrive within
+ *     The boundary for a session is defined by a {@code sessionGap}. All messages that that arrive within
  *     the gap are grouped into the same session.
  *   <li>
  *     Global Windows: A global window defines a single infinite window over the entire {@link org.apache.samza.operators.MessageStream}.
  *     An early trigger must be specified when defining a global window.
  * </ul>
  *
- * <p> A {@link Window} is defined as "keyed" when the incoming {@link MessageEnvelope}s are first grouped based on their key
+ * <p> A {@link Window} is defined as "keyed" when the incoming messages are first grouped based on their key
  * and triggers are fired and window panes are emitted per-key. It is possible to construct "keyed" variants of all the above window
  * types.
  *
@@ -92,7 +91,7 @@ public final class Windows {
   private Windows() { }
 
   /**
-   * Creates a {@link Window} that groups incoming {@link MessageEnvelope}s into fixed-size, non-overlapping processing
+   * Creates a {@link Window} that groups incoming messages into fixed-size, non-overlapping processing
    * time based windows based on the provided keyFn and applies the provided fold function to them.
    *
    * <p>The below example computes the maximum value per-key over fixed size 10 second windows.
@@ -101,29 +100,29 @@ public final class Windows {
    *    MessageStream<UserClick> stream = ...;
    *    Function<UserClick, String> keyFn = ...;
    *    BiFunction<UserClick, Integer, Integer> maxAggregator = (m, c)-> Math.max(parseInt(m), c);
-   *    MessageStream<WindowOutput<WindowKey<String>, Integer>> windowedStream = stream.window(
-   *    Windows.keyedTumblingWindow(keyFn, Duration.ofSeconds(10), maxAggregator));
+   *    MessageStream<WindowPane<String, Integer>> windowedStream = stream.window(
+   *        Windows.keyedTumblingWindow(keyFn, Duration.ofSeconds(10), maxAggregator));
    * }
    * </pre>
    *
-   * @param keyFn the function to extract the window key from a {@link MessageEnvelope}
+   * @param keyFn the function to extract the window key from a message
    * @param interval the duration in processing time
-   * @param foldFn the function to aggregate {@link MessageEnvelope}s in the {@link WindowPane}
-   * @param <M> the type of the input {@link MessageEnvelope}
+   * @param foldFn the function to aggregate messages in the {@link WindowPane}
+   * @param <M> the type of the input message
    * @param <WV> the type of the {@link WindowPane} output value
    * @param <K> the type of the key in the {@link Window}
    * @return the created {@link Window} function.
    */
-  public static <M extends MessageEnvelope, K, WV> Window<M, K, WV, WindowPane<K, WV>>
+  public static <M, K, WV> Window<M, K, WV>
     keyedTumblingWindow(Function<M, K> keyFn, Duration interval, BiFunction<M, WV, WV> foldFn) {
 
-    Trigger defaultTrigger = new TimeTrigger(interval);
+    Trigger<M> defaultTrigger = new TimeTrigger<>(interval);
     return new WindowInternal<M, K, WV>(defaultTrigger, foldFn, keyFn, null);
   }
 
 
   /**
-   * Creates a {@link Window} that groups incoming {@link MessageEnvelope}s into fixed-size, non-overlapping
+   * Creates a {@link Window} that groups incoming messages into fixed-size, non-overlapping
    * processing time based windows using the provided keyFn.
    *
    * <p>The below example groups the stream into fixed-size 10 second windows for each key.
@@ -131,19 +130,18 @@ public final class Windows {
    * <pre> {@code
    *    MessageStream<UserClick> stream = ...;
    *    Function<UserClick, String> keyFn = ...;
-   *    MessageStream<WindowOutput<WindowKey<String>, Collection<UserClick>>> windowedStream = stream.window(
-   *    Windows.keyedTumblingWindow(keyFn, Duration.ofSeconds(10)));
+   *    MessageStream<WindowPane<String, Collection<UserClick>>> windowedStream = stream.window(
+   *        Windows.keyedTumblingWindow(keyFn, Duration.ofSeconds(10)));
    * }
    * </pre>
    *
-   * @param keyFn function to extract key from the {@link MessageEnvelope}
+   * @param keyFn function to extract key from the message
    * @param interval the duration in processing time
-   * @param <M> the type of the input {@link MessageEnvelope}
+   * @param <M> the type of the input message
    * @param <K> the type of the key in the {@link Window}
    * @return the created {@link Window} function
    */
-  public static <M extends MessageEnvelope, K> Window<M, K, Collection<M>, WindowPane<K, Collection<M>>>
-    keyedTumblingWindow(Function<M, K> keyFn, Duration interval) {
+  public static <M, K> Window<M, K, Collection<M>> keyedTumblingWindow(Function<M, K> keyFn, Duration interval) {
     BiFunction<M, Collection<M>, Collection<M>> aggregator = (m, c) -> {
       c.add(m);
       return c;
@@ -160,25 +158,25 @@ public final class Windows {
    * <pre> {@code
    *    MessageStream<String> stream = ...;
    *    BiFunction<String, Integer, Integer> maxAggregator = (m, c)-> Math.max(parseInt(m), c);
-   *    MessageStream<WindowOutput<WindowKey, Integer>> windowedStream = stream.window(
-   *    Windows.tumblingWindow(Duration.ofSeconds(10), maxAggregator));
+   *    MessageStream<WindowPane<Void, Integer>> windowedStream = stream.window(
+   *        Windows.tumblingWindow(Duration.ofSeconds(10), maxAggregator));
    * }
    * </pre>
    *
    * @param duration the duration in processing time
-   * @param foldFn to aggregate {@link MessageEnvelope}s in the {@link WindowPane}
-   * @param <M> the type of the input {@link MessageEnvelope}
+   * @param foldFn to aggregate messages in the {@link WindowPane}
+   * @param <M> the type of the input message
    * @param <WV> the type of the {@link WindowPane} output value
    * @return the created {@link Window} function
    */
-  public static <M extends MessageEnvelope, WV> Window<M, Void, WV, WindowPane<Void, WV>>
+  public static <M, WV> Window<M, Void, WV>
     tumblingWindow(Duration duration, BiFunction<M, WV, WV> foldFn) {
-    Trigger defaultTrigger = Triggers.repeat(new TimeTrigger(duration));
-    return new WindowInternal<M, Void, WV>(defaultTrigger, foldFn, null, null);
+    Trigger<M> defaultTrigger = Triggers.repeat(new TimeTrigger<>(duration));
+    return new WindowInternal<>(defaultTrigger, foldFn, null, null);
   }
 
   /**
-   * Creates a {@link Window} that groups incoming {@link MessageEnvelope}s into fixed-size, non-overlapping
+   * Creates a {@link Window} that groups incoming messages into fixed-size, non-overlapping
    * processing time based windows.
    *
    * <p>The below example groups the stream into fixed-size 10 second windows and computes a windowed-percentile.
@@ -187,16 +185,16 @@ public final class Windows {
    *    MessageStream<Long> stream = ...;
    *    Function<Collection<Long, Long>> percentile99 = ..
    *
-   *    MessageStream<WindowOutput<WindowKey, Collection<Long>>> windowedStream = integerStream.window(Windows.tumblingWindow(Duration.ofSeconds(10)));
+   *    MessageStream<WindowPane<Void, Collection<Long>>> windowedStream = integerStream.window(Windows.tumblingWindow(Duration.ofSeconds(10)));
    *    MessageStream<Long> windowedPercentiles = windowed.map(windowedOutput -> percentile99(windowedOutput.getMessage());
    * }
    * </pre>
    *
    * @param duration the duration in processing time
-   * @param <M> the type of the input {@link MessageEnvelope}
+   * @param <M> the type of the input message
    * @return the created {@link Window} function
    */
-  public static <M extends MessageEnvelope> Window<M, Void, Collection<M>, WindowPane<Void, Collection<M>>> tumblingWindow(Duration duration) {
+  public static <M> Window<M, Void, Collection<M>> tumblingWindow(Duration duration) {
     BiFunction<M, Collection<M>, Collection<M>> aggregator = (m, c) -> {
       c.add(m);
       return c;
@@ -205,11 +203,11 @@ public final class Windows {
   }
 
   /**
-   * Creates a {@link Window} that groups incoming {@link MessageEnvelope}s into sessions per-key based on the provided {@code sessionGap}
+   * Creates a {@link Window} that groups incoming messages into sessions per-key based on the provided {@code sessionGap}
    * and applies the provided fold function to them.
    *
    * <p>A <i>session</i> captures some period of activity over a {@link org.apache.samza.operators.MessageStream}.
-   * A session is considered complete when no new messages arrive within the {@code sessionGap}. All {@link MessageEnvelope}s that arrive within
+   * A session is considered complete when no new messages arrive within the {@code sessionGap}. All messages that arrive within
    * the gap are grouped into the same session.
    *
    * <p>The below example computes the maximum value per-key over a session window of gap 10 seconds.
@@ -218,29 +216,29 @@ public final class Windows {
    *    MessageStream<UserClick> stream = ...;
    *    BiFunction<UserClick, Integer, Integer> maxAggregator = (m, c)-> Math.max(parseInt(m), c);
    *    Function<UserClick, String> userIdExtractor = m -> m.getUserId()..;
-   *    MessageStream<WindowOutput<WindowKey<String>, Integer>> windowedStream = stream.window(
-   *    Windows.keyedSessionWindow(userIdExtractor, Duration.minute(1), maxAggregator));
+   *    MessageStream<WindowPane<String, Integer>> windowedStream = stream.window(
+   *        Windows.keyedSessionWindow(userIdExtractor, Duration.minute(1), maxAggregator));
    * }
    * </pre>
    *
-   * @param keyFn the function to extract the window key from a {@link MessageEnvelope}
+   * @param keyFn the function to extract the window key from a message
    * @param sessionGap the timeout gap for defining the session
-   * @param foldFn the function to aggregate {@link MessageEnvelope}s in the {@link WindowPane}
-   * @param <M> the type of the input {@link MessageEnvelope}
+   * @param foldFn the function to aggregate messages in the {@link WindowPane}
+   * @param <M> the type of the input message
    * @param <K> the type of the key in the {@link Window}
    * @param <WV> the type of the output value in the {@link WindowPane}
    * @return the created {@link Window} function
    */
-  public static <M extends MessageEnvelope, K, WV> Window<M, K, WV, WindowPane<K, WV>> keyedSessionWindow(Function<M, K> keyFn, Duration sessionGap, BiFunction<M, WV, WV> foldFn) {
-    Trigger defaultTrigger = Triggers.timeSinceLastMessage(sessionGap);
-    return new WindowInternal<M, K, WV>(defaultTrigger, foldFn, keyFn, null);
+  public static <M, K, WV> Window<M, K, WV> keyedSessionWindow(Function<M, K> keyFn, Duration sessionGap, BiFunction<M, WV, WV> foldFn) {
+    Trigger<M> defaultTrigger = Triggers.timeSinceLastMessage(sessionGap);
+    return new WindowInternal<>(defaultTrigger, foldFn, keyFn, null);
   }
 
   /**
-   * Creates a {@link Window} that groups incoming {@link MessageEnvelope}s into sessions per-key based on the provided {@code sessionGap}.
+   * Creates a {@link Window} that groups incoming messages into sessions per-key based on the provided {@code sessionGap}.
    *
    * <p>A <i>session</i> captures some period of activity over a {@link org.apache.samza.operators.MessageStream}. The
-   * boundary for the session is defined by a {@code sessionGap}. All {@link MessageEnvelope}s that that arrive within
+   * boundary for the session is defined by a {@code sessionGap}. All messages that that arrive within
    * the gap are grouped into the same session.
    *
    * <p>The below example groups the stream into per-key session windows of gap 10 seconds.
@@ -249,18 +247,18 @@ public final class Windows {
    *    MessageStream<UserClick> stream = ...;
    *    BiFunction<UserClick, Integer, Integer> maxAggregator = (m, c)-> Math.max(parseIntField(m), c);
    *    Function<UserClick, String> userIdExtractor = m -> m.getUserId()..;
-   *    MessageStream<WindowOutput<WindowKey<String>, Collection<M>>> windowedStream = stream.window(
-   *    Windows.keyedSessionWindow(userIdExtractor, Duration.ofSeconds(10)));
+   *    MessageStream<WindowPane<String>, Collection<M>> windowedStream = stream.window(
+   *        Windows.keyedSessionWindow(userIdExtractor, Duration.ofSeconds(10)));
    * }
    * </pre>
    *
-   * @param keyFn the function to extract the window key from a {@link MessageEnvelope}
+   * @param keyFn the function to extract the window key from a message}
    * @param sessionGap the timeout gap for defining the session
-   * @param <M> the type of the input {@link MessageEnvelope}
+   * @param <M> the type of the input message
    * @param <K> the type of the key in the {@link Window}
    * @return the created {@link Window} function
    */
-  public static <M extends MessageEnvelope, K> Window<M, K, Collection<M>, WindowPane<K, Collection<M>>> keyedSessionWindow(Function<M, K> keyFn, Duration sessionGap) {
+  public static <M, K> Window<M, K, Collection<M>> keyedSessionWindow(Function<M, K> keyFn, Duration sessionGap) {
 
     BiFunction<M, Collection<M>, Collection<M>> aggregator = (m, c) -> {
       c.add(m);
@@ -271,7 +269,7 @@ public final class Windows {
 
 
   /**
-   * Creates a {@link Window} that groups incoming {@link MessageEnvelope}s into a single global window. This window does not have a
+   * Creates a {@link Window} that groups incoming messages into a single global window. This window does not have a
    * default trigger. The triggering behavior must be specified by setting an early trigger.
    *
    * <p>The below example computes the maximum value over a count based window. The window emits {@link WindowPane}s when
@@ -280,36 +278,36 @@ public final class Windows {
    * <pre> {@code
    *    MessageStream<Long> stream = ...;
    *    BiFunction<Long, Long, Long> maxAggregator = (m, c)-> Math.max(m, c);
-   *    MessageStream<WindowOutput<WindowKey, Long>> windowedStream = stream.window(Windows.globalWindow(maxAggregator)
+   *    MessageStream<WindowPane<Void, Long>> windowedStream = stream.window(Windows.globalWindow(maxAggregator)
    *      .setEarlyTriggers(Triggers.repeat(Triggers.any(Triggers.count(50), Triggers.timeSinceFirstMessage(Duration.ofSeconds(10))))))
    * }
    * </pre>
    *
-   * @param foldFn the function to aggregate {@link MessageEnvelope}s in the {@link WindowPane}
-   * @param <M> the type of {@link MessageEnvelope}
+   * @param foldFn the function to aggregate messages in the {@link WindowPane}
+   * @param <M> the type of message
    * @param <WV> type of the output value in the {@link WindowPane}
    * @return the created {@link Window} function.
    */
-  public static <M extends MessageEnvelope, WV> Window<M, Void, WV, WindowPane<Void, WV>> globalWindow(BiFunction<M, WV, WV> foldFn) {
-    return new WindowInternal<M, Void, WV>(null, foldFn, null, null);
+  public static <M, WV> Window<M, Void, WV> globalWindow(BiFunction<M, WV, WV> foldFn) {
+    return new WindowInternal<>(null, foldFn, null, null);
   }
 
   /**
-   * Creates a {@link Window} that groups incoming {@link MessageEnvelope}s into a single global window. This window does not have a
+   * Creates a {@link Window} that groups incoming messages into a single global window. This window does not have a
    * default trigger. The triggering behavior must be specified by setting an early trigger.
    *
    * The below example groups the stream into count based windows that trigger every 50 messages or every 10 minutes.
    * <pre> {@code
    *    MessageStream<Long> stream = ...;
-   *    MessageStream<WindowOutput<WindowKey, Collection<Long>>> windowedStream = stream.window(Windows.globalWindow()
+   *    MessageStream<WindowPane<Void, Collection<Long>> windowedStream = stream.window(Windows.globalWindow()
    *      .setEarlyTrigger(Triggers.repeat(Triggers.any(Triggers.count(50), Triggers.timeSinceFirstMessage(Duration.ofSeconds(10))))))
    * }
    * </pre>
    *
-   * @param <M> the type of {@link MessageEnvelope}
+   * @param <M> the type of message
    * @return the created {@link Window} function.
    */
-  public static <M extends MessageEnvelope> Window<M, Void, Collection<M>, WindowPane<Void, Collection<M>>> globalWindow() {
+  public static <M> Window<M, Void, Collection<M>> globalWindow() {
     BiFunction<M, Collection<M>, Collection<M>> aggregator = (m, c) -> {
       c.add(m);
       return c;
@@ -318,7 +316,7 @@ public final class Windows {
   }
 
   /**
-   * Returns a global {@link Window} that groups incoming {@link MessageEnvelope}s using the provided keyFn.
+   * Returns a global {@link Window} that groups incoming messages using the provided keyFn.
    * The window does not have a default trigger. The triggering behavior must be specified by setting an early
    * trigger.
    *
@@ -329,24 +327,24 @@ public final class Windows {
    *    MessageStream<UserClick> stream = ...;
    *    BiFunction<UserClick, Long, Long> maxAggregator = (m, c)-> Math.max(parseLongField(m), c);
    *    Function<UserClick, String> keyFn = ...;
-   *    MessageStream<WindowOutput<WindowKey<String>, Long>> windowedStream = stream.window(Windows.keyedGlobalWindow(keyFn, maxAggregator)
+   *    MessageStream<WindowPane<String, Long>> windowedStream = stream.window(Windows.keyedGlobalWindow(keyFn, maxAggregator)
    *      .setEarlyTrigger(Triggers.repeat(Triggers.any(Triggers.count(50), Triggers.timeSinceFirstMessage(Duration.minutes(10))))))
    * }
    * </pre>
    *
-   * @param keyFn the function to extract the window key from a {@link MessageEnvelope}
-   * @param foldFn the function to aggregate {@link MessageEnvelope}s in the {@link WindowPane}
-   * @param <M> the type of {@link MessageEnvelope}
+   * @param keyFn the function to extract the window key from a message
+   * @param foldFn the function to aggregate messages in the {@link WindowPane}
+   * @param <M> the type of message
    * @param <K> type of the key in the {@link Window}
    * @param <WV> the type of the output value in the {@link WindowPane}
    * @return the created {@link Window} function
    */
-  public static <M extends MessageEnvelope, K, WV> Window<M, K, WV, WindowPane<K, WV>> keyedGlobalWindow(Function<M, K> keyFn, BiFunction<M, WV, WV> foldFn) {
+  public static <M, K, WV> Window<M, K, WV> keyedGlobalWindow(Function<M, K> keyFn, BiFunction<M, WV, WV> foldFn) {
     return new WindowInternal<M, K, WV>(null, foldFn, keyFn, null);
   }
 
   /**
-   * Returns a global {@link Window} that groups incoming {@link MessageEnvelope}s using the provided keyFn.
+   * Returns a global {@link Window} that groups incoming messages using the provided keyFn.
    * The window does not have a default trigger. The triggering behavior must be specified by setting an early trigger.
    *
    * <p> The below example groups the stream per-key into count based windows. The window triggers every 50 messages or
@@ -355,17 +353,17 @@ public final class Windows {
    * <pre> {@code
    *    MessageStream<UserClick> stream = ...;
    *    Function<UserClick, String> keyFn = ...;
-   *    MessageStream<WindowOutput<WindowKey<String>, Collection<UserClick>>> windowedStream = stream.window(Windows.keyedGlobalWindow(keyFn)
+   *    MessageStream<WindowPane<String, Collection<UserClick>> windowedStream = stream.window(Windows.keyedGlobalWindow(keyFn)
    *      .setEarlyTrigger(Triggers.repeat(Triggers.any(Triggers.count(50), Triggers.timeSinceFirstMessage(Duration.minutes(10))))))
    * }
    * </pre>
    *
-   * @param keyFn the function to extract the window key from a {@link MessageEnvelope}
-   * @param <M> the type of {@link MessageEnvelope}
+   * @param keyFn the function to extract the window key from a message
+   * @param <M> the type of message
    * @param <K> the type of the key in the {@link Window}
    * @return the created {@link Window} function
    */
-  public static <M extends MessageEnvelope, K> Window<M, K, Collection<M>, WindowPane<K, Collection<M>>> keyedGlobalWindow(Function<M, K> keyFn) {
+  public static <M, K> Window<M, K, Collection<M>> keyedGlobalWindow(Function<M, K> keyFn) {
     BiFunction<M, Collection<M>, Collection<M>> aggregator = (m, c) -> {
       c.add(m);
       return c;

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/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 8825867..9479eea 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,11 +18,9 @@
  */
 package org.apache.samza.operators.windows.internal;
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.MessageEnvelope;
 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.operators.windows.WindowPane;
 
 import java.util.function.BiFunction;
 import java.util.function.Function;
@@ -32,9 +30,13 @@ import java.util.function.Function;
  *  and whether to accumulate or discard previously emitted panes.
  *
  *  Note: This class is meant to be used internally by Samza, and is not to be instantiated by programmers.
+ *
+ * @param <M>  the type of input message
+ * @param <K>  the type of key for the window
+ * @param <WV>  the type of aggregated value in the window output
  */
 @InterfaceStability.Unstable
-public final class WindowInternal<M extends MessageEnvelope, K, WV> implements Window<M, K, WV, WindowPane<K, WV>> {
+public final class WindowInternal<M, K, WV> implements Window<M, K, WV> {
 
   private final Trigger defaultTrigger;
 
@@ -67,19 +69,19 @@ public final class WindowInternal<M extends MessageEnvelope, K, WV> implements W
   }
 
   @Override
-  public Window<M, K, WV, WindowPane<K, WV>> setEarlyTrigger(Trigger trigger) {
+  public Window<M, K, WV> setEarlyTrigger(Trigger trigger) {
     this.earlyTrigger = trigger;
     return this;
   }
 
   @Override
-  public Window<M, K, WV, WindowPane<K, WV>> setLateTrigger(Trigger trigger) {
+  public Window<M, K, WV> setLateTrigger(Trigger trigger) {
     this.lateTrigger = trigger;
     return this;
   }
 
   @Override
-  public Window<M, K, WV, WindowPane<K, WV>> setAccumulationMode(AccumulationMode mode) {
+  public Window<M, K, WV> setAccumulationMode(AccumulationMode mode) {
     this.mode = mode;
     return this;
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/main/java/org/apache/samza/system/ExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/ExecutionEnvironment.java b/samza-api/src/main/java/org/apache/samza/system/ExecutionEnvironment.java
new file mode 100644
index 0000000..d0c5985
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/ExecutionEnvironment.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.config.ConfigException;
+import org.apache.samza.operators.StreamGraphFactory;
+import org.apache.samza.config.Config;
+
+
+/**
+ * Interface to be implemented by physical execution engine to deploy the config and jobs to run the {@link org.apache.samza.operators.StreamGraph}
+ */
+@InterfaceStability.Unstable
+public interface ExecutionEnvironment {
+
+  String ENVIRONMENT_CONFIG = "job.execution.environment.class";
+  String DEFAULT_ENVIRONMENT_CLASS = "org.apache.samza.system.StandaloneExecutionEnvironment";
+
+  /**
+   * Static method to load the local standalone environment
+   *
+   * @param config  configuration passed in to initialize the Samza standalone process
+   * @return  the standalone {@link ExecutionEnvironment} to run the user-defined stream applications
+   */
+  static ExecutionEnvironment getLocalEnvironment(Config config) {
+    return null;
+  }
+
+  /**
+   * Static method to load the non-standalone environment.
+   *
+   * @param config  configuration passed in to initialize the Samza processes
+   * @return  the configure-driven {@link ExecutionEnvironment} to run the user-defined stream applications
+   */
+  static ExecutionEnvironment fromConfig(Config config) {
+    try {
+      if (ExecutionEnvironment.class.isAssignableFrom(Class.forName(config.get(ENVIRONMENT_CONFIG, DEFAULT_ENVIRONMENT_CLASS)))) {
+        return (ExecutionEnvironment) Class.forName(config.get(ENVIRONMENT_CONFIG, DEFAULT_ENVIRONMENT_CLASS)).newInstance();
+      }
+    } catch (Exception e) {
+      throw new ConfigException(String.format("Problem in loading ExecutionEnvironment class %s", config.get(ENVIRONMENT_CONFIG)), e);
+    }
+    throw new ConfigException(String.format(
+        "Class %s does not implement interface ExecutionEnvironment properly",
+        config.get(ENVIRONMENT_CONFIG)));
+  }
+
+  /**
+   * Method to be invoked to deploy and run the actual Samza jobs to execute {@link org.apache.samza.operators.StreamGraph}
+   *
+   * @param graphFactory  the user-defined {@link StreamGraphFactory} object
+   * @param config  the {@link Config} object for this job
+   */
+  void run(StreamGraphFactory graphFactory, Config config);
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/main/java/org/apache/samza/task/TaskContext.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/task/TaskContext.java b/samza-api/src/main/java/org/apache/samza/task/TaskContext.java
index a85e0b4..5779071 100644
--- a/samza-api/src/main/java/org/apache/samza/task/TaskContext.java
+++ b/samza-api/src/main/java/org/apache/samza/task/TaskContext.java
@@ -54,4 +54,14 @@ public interface TaskContext {
    *
    */
   void setStartingOffset(SystemStreamPartition ssp, String offset);
+
+  /**
+   * Method to allow user to return customized context
+   *
+   * @param <T>  the type of user-defined task context
+   * @return  user-defined task context object
+   */
+  default <T extends TaskContext> T getUserDefinedContext() {
+    return null;
+  };
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/test/java/org/apache/samza/operators/data/TestIncomingSystemMessage.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/data/TestIncomingSystemMessage.java b/samza-api/src/test/java/org/apache/samza/operators/data/TestIncomingSystemMessage.java
index 7bd62a7..e3a1290 100644
--- a/samza-api/src/test/java/org/apache/samza/operators/data/TestIncomingSystemMessage.java
+++ b/samza-api/src/test/java/org/apache/samza/operators/data/TestIncomingSystemMessage.java
@@ -33,7 +33,7 @@ public class TestIncomingSystemMessage {
   @Test
   public void testConstructor() {
     IncomingMessageEnvelope ime = mock(IncomingMessageEnvelope.class);
-    IncomingSystemMessageEnvelope ism = new IncomingSystemMessageEnvelope(ime);
+    InputMessageEnvelope ism = new InputMessageEnvelope(ime);
 
     Object mockKey = mock(Object.class);
     Object mockValue = mock(Object.class);

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowOutput.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowOutput.java b/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowOutput.java
deleted file mode 100644
index 9679e1d..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowOutput.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators.windows;
-
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-
-
-public class TestWindowOutput {
-  @Test
-  public void testConstructor() {
-    WindowPane<String, Integer> wndOutput = WindowPane.of(new WindowKey("testMsg", null), 10);
-    assertEquals(wndOutput.getKey().getKey(), "testMsg");
-    assertEquals(wndOutput.getMessage(), Integer.valueOf(10));
-    assertFalse(wndOutput.isDelete());
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/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
new file mode 100644
index 0000000..809c5b4
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowPane.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.windows;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+
+public class TestWindowPane {
+  @Test
+  public void testConstructor() {
+    WindowPane<String, Integer> wndOutput = WindowPane.of(new WindowKey("testMsg", null), 10);
+    assertEquals(wndOutput.getKey().getKey(), "testMsg");
+    assertEquals(wndOutput.getMessage(), Integer.valueOf(10));
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
index 286893c..d85d488 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
@@ -19,107 +19,161 @@
 
 package org.apache.samza.operators;
 
-import org.apache.samza.operators.data.MessageEnvelope;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.function.Function;
+import java.util.HashSet;
+import java.util.Set;
+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.JoinFunction;
 import org.apache.samza.operators.functions.MapFunction;
+import org.apache.samza.operators.functions.PartialJoinFunction;
 import org.apache.samza.operators.functions.SinkFunction;
 import org.apache.samza.operators.spec.OperatorSpec;
 import org.apache.samza.operators.spec.OperatorSpecs;
 import org.apache.samza.operators.windows.Window;
 import org.apache.samza.operators.windows.WindowPane;
 import org.apache.samza.operators.windows.internal.WindowInternal;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.function.BiFunction;
+import org.apache.samza.task.TaskContext;
 
 
 /**
  * The implementation for input/output {@link MessageStream}s to/from the operators.
  * Users use the {@link MessageStream} API methods to describe and chain the operators specs.
  *
- * @param <M>  type of {@link MessageEnvelope}s in this {@link MessageStream}
+ * @param <M>  type of messages in this {@link MessageStream}
  */
-public class MessageStreamImpl<M extends MessageEnvelope> implements MessageStream<M> {
+public class MessageStreamImpl<M> implements MessageStream<M> {
+  /**
+   * The {@link StreamGraphImpl} object that contains this {@link MessageStreamImpl}
+   */
+  private final StreamGraphImpl graph;
 
   /**
-   * The set of operators that consume the {@link MessageEnvelope}s in this {@link MessageStream}
+   * The set of operators that consume the messages in this {@link MessageStream}
    */
   private final Set<OperatorSpec> registeredOperatorSpecs = new HashSet<>();
 
-  @Override
-  public <OM extends MessageEnvelope> MessageStream<OM> map(MapFunction<M, OM> mapFn) {
-    OperatorSpec<OM> op = OperatorSpecs.<M, OM>createStreamOperatorSpec(m -> new ArrayList<OM>() { {
-        OM r = mapFn.apply(m);
-        if (r != null) {
-          this.add(r);
-        }
-      } });
+  /**
+   * Default constructor
+   *
+   * @param graph the {@link StreamGraphImpl} object that this stream belongs to
+   */
+  MessageStreamImpl(StreamGraphImpl graph) {
+    this.graph = graph;
+  }
+
+  @Override public <TM> MessageStream<TM> map(MapFunction<M, TM> mapFn) {
+    OperatorSpec<TM> op = OperatorSpecs.<M, TM>createMapOperatorSpec(mapFn, this.graph, new MessageStreamImpl<>(this.graph));
     this.registeredOperatorSpecs.add(op);
-    return op.getOutputStream();
+    return op.getNextStream();
   }
 
-  @Override
-  public <OM extends MessageEnvelope> MessageStream<OM> flatMap(FlatMapFunction<M, OM> flatMapFn) {
-    OperatorSpec<OM> op = OperatorSpecs.createStreamOperatorSpec(flatMapFn);
+  @Override public MessageStream<M> filter(FilterFunction<M> filterFn) {
+    OperatorSpec<M> op = OperatorSpecs.<M>createFilterOperatorSpec(filterFn, this.graph, new MessageStreamImpl<>(this.graph));
     this.registeredOperatorSpecs.add(op);
-    return op.getOutputStream();
+    return op.getNextStream();
   }
 
   @Override
-  public MessageStream<M> filter(FilterFunction<M> filterFn) {
-    OperatorSpec<M> op = OperatorSpecs.<M, M>createStreamOperatorSpec(t -> new ArrayList<M>() { {
-        if (filterFn.apply(t)) {
-          this.add(t);
-        }
-      } });
+  public <TM> MessageStream<TM> flatMap(FlatMapFunction<M, TM> flatMapFn) {
+    OperatorSpec<TM> op = OperatorSpecs.createStreamOperatorSpec(flatMapFn, this.graph, new MessageStreamImpl<>(this.graph));
     this.registeredOperatorSpecs.add(op);
-    return op.getOutputStream();
+    return op.getNextStream();
   }
 
   @Override
   public void sink(SinkFunction<M> sinkFn) {
-    this.registeredOperatorSpecs.add(OperatorSpecs.createSinkOperatorSpec(sinkFn));
+    this.registeredOperatorSpecs.add(OperatorSpecs.createSinkOperatorSpec(sinkFn, this.graph));
   }
 
-  @Override
-  public <K, WV, WM extends WindowPane<K, WV>> MessageStream<WM> window(
-      Window<M, K, WV, WM> window) {
-    OperatorSpec<WM> wndOp = OperatorSpecs.createWindowOperatorSpec((WindowInternal<MessageEnvelope, K, WV>) window);
-    this.registeredOperatorSpecs.add(wndOp);
-    return wndOp.getOutputStream();
+  @Override public void sendTo(OutputStream<M> stream) {
+    this.registeredOperatorSpecs.add(OperatorSpecs.createSendToOperatorSpec(stream.getSinkFunction(), this.graph, stream));
+  }
+
+  @Override public MessageStream<M> sendThrough(OutputStream<M> stream) {
+    this.sendTo(stream);
+    return this.graph.getIntStream(stream);
   }
 
   @Override
-  public <K, JM extends MessageEnvelope<K, ?>, RM extends MessageEnvelope> MessageStream<RM> join(
-      MessageStream<JM> otherStream, JoinFunction<M, JM, RM> joinFn) {
-    MessageStreamImpl<RM> outputStream = new MessageStreamImpl<>();
+  public <K, WV> MessageStream<WindowPane<K, WV>> window(Window<M, K, WV> window) {
+    OperatorSpec<WindowPane<K, WV>> wndOp = OperatorSpecs.createWindowOperatorSpec((WindowInternal<M, K, WV>) window,
+        this.graph, new MessageStreamImpl<>(this.graph));
+    this.registeredOperatorSpecs.add(wndOp);
+    return wndOp.getNextStream();
+  }
 
-    BiFunction<M, JM, RM> parJoin1 = joinFn::apply;
-    BiFunction<JM, M, RM> parJoin2 = (m, t1) -> joinFn.apply(t1, m);
+  @Override public <K, OM, RM> MessageStream<RM> join(MessageStream<OM> otherStream, JoinFunction<K, M, OM, RM> joinFn) {
+    MessageStreamImpl<RM> outputStream = new MessageStreamImpl<>(this.graph);
+
+    PartialJoinFunction<K, M, OM, RM> parJoin1 = new PartialJoinFunction<K, M, OM, RM>() {
+      @Override
+      public RM apply(M m1, OM om) {
+        return joinFn.apply(m1, om);
+      }
+
+      @Override
+      public K getKey(M message) {
+        return joinFn.getFirstKey(message);
+      }
+
+      @Override
+      public K getOtherKey(OM message) {
+        return joinFn.getSecondKey(message);
+      }
+
+      @Override
+      public void init(Config config, TaskContext context) {
+        joinFn.init(config, context);
+      }
+    };
+
+    PartialJoinFunction<K, OM, M, RM> parJoin2 = new PartialJoinFunction<K, OM, M, RM>() {
+      @Override
+      public RM apply(OM m1, M m) {
+        return joinFn.apply(m, m1);
+      }
+
+      @Override
+      public K getKey(OM message) {
+        return joinFn.getSecondKey(message);
+      }
+
+      @Override
+      public K getOtherKey(M message) {
+        return joinFn.getFirstKey(message);
+      }
+    };
 
     // TODO: need to add default store functions for the two partial join functions
 
-    ((MessageStreamImpl<JM>) otherStream).registeredOperatorSpecs.add(OperatorSpecs.createPartialJoinOperatorSpec(parJoin2, outputStream));
-    this.registeredOperatorSpecs.add(OperatorSpecs.createPartialJoinOperatorSpec(parJoin1, outputStream));
+    ((MessageStreamImpl<OM>) otherStream).registeredOperatorSpecs.add(
+        OperatorSpecs.<OM, K, M, RM>createPartialJoinOperatorSpec(parJoin2, this.graph, outputStream));
+    this.registeredOperatorSpecs.add(OperatorSpecs.<M, K, OM, RM>createPartialJoinOperatorSpec(parJoin1, this.graph, outputStream));
     return outputStream;
   }
 
   @Override
   public MessageStream<M> merge(Collection<MessageStream<M>> otherStreams) {
-    MessageStreamImpl<M> outputStream = new MessageStreamImpl<>();
+    MessageStreamImpl<M> outputStream = new MessageStreamImpl<>(this.graph);
 
     otherStreams.add(this);
-    otherStreams.forEach(other ->
-        ((MessageStreamImpl<M>) other).registeredOperatorSpecs.add(OperatorSpecs.createMergeOperatorSpec(outputStream)));
+    otherStreams.forEach(other -> ((MessageStreamImpl<M>) other).registeredOperatorSpecs.
+        add(OperatorSpecs.createMergeOperatorSpec(this.graph, outputStream)));
     return outputStream;
   }
 
+  @Override
+  public <K> MessageStream<M> partitionBy(Function<M, K> parKeyExtractor) {
+    MessageStreamImpl<M> intStream = this.graph.createIntStream(parKeyExtractor);
+    OutputStream<M> outputStream = this.graph.getOutputStream(intStream);
+    this.registeredOperatorSpecs.add(OperatorSpecs.createPartitionOperatorSpec(outputStream.getSinkFunction(),
+        this.graph, outputStream));
+    return intStream;
+  }
   /**
    * Gets the operator specs registered to consume the output of this {@link MessageStream}. This is an internal API and
    * should not be exposed to users.
@@ -129,4 +183,5 @@ public class MessageStreamImpl<M extends MessageEnvelope> implements MessageStre
   public Collection<OperatorSpec> getRegisteredOperatorSpecs() {
     return Collections.unmodifiableSet(this.registeredOperatorSpecs);
   }
+
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/StreamGraphImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
new file mode 100644
index 0000000..dca3469
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
@@ -0,0 +1,260 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators;
+
+import java.util.Properties;
+import java.util.function.Function;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * The implementation of {@link StreamGraph} interface. This class provides implementation of methods to allow users to
+ * create system input/output/intermediate streams.
+ */
+public class StreamGraphImpl implements StreamGraph {
+
+  /**
+   * Unique identifier for each {@link org.apache.samza.operators.spec.OperatorSpec} added to transform the {@link MessageEnvelope}
+   * in the input {@link MessageStream}s.
+   */
+  private int opId = 0;
+
+  private class InputStreamImpl<K, V, M extends MessageEnvelope<K, V>> extends MessageStreamImpl<M> {
+    final StreamSpec spec;
+    final Serde<K> keySerde;
+    final Serde<V> msgSerde;
+
+    InputStreamImpl(StreamGraphImpl graph, StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
+      super(graph);
+      this.spec = streamSpec;
+      this.keySerde = keySerde;
+      this.msgSerde = msgSerde;
+    }
+
+    StreamSpec getSpec() {
+      return this.spec;
+    }
+
+  }
+
+  private class OutputStreamImpl<K, V, M extends MessageEnvelope<K, V>> implements OutputStream<M> {
+    final StreamSpec spec;
+    final Serde<K> keySerde;
+    final Serde<V> msgSerde;
+
+    OutputStreamImpl(StreamGraphImpl graph, StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
+      this.spec = streamSpec;
+      this.keySerde = keySerde;
+      this.msgSerde = msgSerde;
+    }
+
+    StreamSpec getSpec() {
+      return this.spec;
+    }
+
+    @Override
+    public SinkFunction<M> getSinkFunction() {
+      return (M message, MessageCollector mc, TaskCoordinator tc) -> {
+        // TODO: need to find a way to directly pass in the serde class names
+        // mc.send(new OutgoingMessageEnvelope(this.spec.getSystemStream(), this.keySerde.getClass().getName(), this.msgSerde.getClass().getName(),
+        //    message.getKey(), message.getKey(), message.getMessage()));
+        mc.send(new OutgoingMessageEnvelope(this.spec.getSystemStream(), message.getKey(), message.getMessage()));
+      };
+    }
+  }
+
+  private class IntermediateStreamImpl<PK, K, V, M extends MessageEnvelope<K, V>> extends InputStreamImpl<K, V, M> implements OutputStream<M> {
+    final Function<M, PK> parKeyFn;
+
+    /**
+     * Default constructor
+     *
+     * @param graph the {@link StreamGraphImpl} object that this stream belongs to
+     */
+    IntermediateStreamImpl(StreamGraphImpl graph, StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
+      this(graph, streamSpec, keySerde, msgSerde, null);
+    }
+
+    IntermediateStreamImpl(StreamGraphImpl graph, StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde, Function<M, PK> parKeyFn) {
+      super(graph, streamSpec, keySerde, msgSerde);
+      this.parKeyFn = parKeyFn;
+    }
+
+    @Override
+    public SinkFunction<M> getSinkFunction() {
+      return (M message, MessageCollector mc, TaskCoordinator tc) -> {
+        // TODO: need to find a way to directly pass in the serde class names
+        // mc.send(new OutgoingMessageEnvelope(this.spec.getSystemStream(), this.keySerde.getClass().getName(), this.msgSerde.getClass().getName(),
+        //    message.getKey(), message.getKey(), message.getMessage()));
+        if (this.parKeyFn == null) {
+          mc.send(new OutgoingMessageEnvelope(this.spec.getSystemStream(), message.getKey(), message.getMessage()));
+        } else {
+          // apply partition key function
+          mc.send(new OutgoingMessageEnvelope(this.spec.getSystemStream(), this.parKeyFn.apply(message), message.getKey(), message.getMessage()));
+        }
+      };
+    }
+  }
+
+  /**
+   * Maps keeping all {@link SystemStream}s that are input and output of operators in {@link StreamGraphImpl}
+   */
+  private final Map<SystemStream, MessageStream> inStreams = new HashMap<>();
+  private final Map<SystemStream, OutputStream> outStreams = new HashMap<>();
+
+  private ContextManager contextManager = new ContextManager() { };
+
+  @Override
+  public <K, V, M extends MessageEnvelope<K, V>> MessageStream<M> createInStream(StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
+    if (!this.inStreams.containsKey(streamSpec.getSystemStream())) {
+      this.inStreams.putIfAbsent(streamSpec.getSystemStream(), new InputStreamImpl<K, V, M>(this, streamSpec, keySerde, msgSerde));
+    }
+    return this.inStreams.get(streamSpec.getSystemStream());
+  }
+
+  /**
+   * Helper method to be used by {@link MessageStreamImpl} class
+   *
+   * @param streamSpec  the {@link StreamSpec} object defining the {@link SystemStream} as the output
+   * @param <M>  the type of {@link MessageEnvelope}s in the output {@link SystemStream}
+   * @return  the {@link MessageStreamImpl} object
+   */
+  @Override
+  public <K, V, M extends MessageEnvelope<K, V>> OutputStream<M> createOutStream(StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
+    if (!this.outStreams.containsKey(streamSpec.getSystemStream())) {
+      this.outStreams.putIfAbsent(streamSpec.getSystemStream(), new OutputStreamImpl<K, V, M>(this, streamSpec, keySerde, msgSerde));
+    }
+    return this.outStreams.get(streamSpec.getSystemStream());
+  }
+
+  /**
+   * Helper method to be used by {@link MessageStreamImpl} class
+   *
+   * @param streamSpec  the {@link StreamSpec} object defining the {@link SystemStream} as an intermediate {@link SystemStream}
+   * @param <M>  the type of {@link MessageEnvelope}s in the output {@link SystemStream}
+   * @return  the {@link MessageStreamImpl} object
+   */
+  @Override
+  public <K, V, M extends MessageEnvelope<K, V>> OutputStream<M> createIntStream(StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
+    if (!this.inStreams.containsKey(streamSpec.getSystemStream())) {
+      this.inStreams.putIfAbsent(streamSpec.getSystemStream(), new IntermediateStreamImpl<K, K, V, M>(this, streamSpec, keySerde, msgSerde));
+    }
+    IntermediateStreamImpl<K, K, V, M> intStream = (IntermediateStreamImpl<K, K, V, M>) this.inStreams.get(streamSpec.getSystemStream());
+    if (!this.outStreams.containsKey(streamSpec.getSystemStream())) {
+      this.outStreams.putIfAbsent(streamSpec.getSystemStream(), intStream);
+    }
+    return intStream;
+  }
+
+  @Override public Map<StreamSpec, MessageStream> getInStreams() {
+    Map<StreamSpec, MessageStream> inStreamMap = new HashMap<>();
+    this.inStreams.forEach((ss, entry) -> inStreamMap.put(((InputStreamImpl) entry).getSpec(), entry));
+    return Collections.unmodifiableMap(inStreamMap);
+  }
+
+  @Override public Map<StreamSpec, OutputStream> getOutStreams() {
+    Map<StreamSpec, OutputStream> outStreamMap = new HashMap<>();
+    this.outStreams.forEach((ss, entry) -> outStreamMap.put(((OutputStreamImpl) entry).getSpec(), entry));
+    return Collections.unmodifiableMap(outStreamMap);
+  }
+
+  @Override
+  public StreamGraph withContextManager(ContextManager manager) {
+    this.contextManager = manager;
+    return this;
+  }
+
+  public int getNextOpId() {
+    return this.opId++;
+  }
+
+  public ContextManager getContextManager() {
+    return this.contextManager;
+  }
+
+  /**
+   * Helper method to be get the input stream via {@link SystemStream}
+   *
+   * @param systemStream  the {@link SystemStream}
+   * @return  a {@link MessageStreamImpl} object corresponding to the {@code systemStream}
+   */
+  public MessageStreamImpl getInputStream(SystemStream systemStream) {
+    if (this.inStreams.containsKey(systemStream)) {
+      return (MessageStreamImpl) this.inStreams.get(systemStream);
+    }
+    return null;
+  }
+
+  <M> OutputStream<M> getOutputStream(MessageStreamImpl<M> intStream) {
+    if (this.outStreams.containsValue(intStream)) {
+      return (OutputStream<M>) intStream;
+    }
+    return null;
+  }
+
+  <M> MessageStream<M> getIntStream(OutputStream<M> outStream) {
+    if (this.inStreams.containsValue(outStream)) {
+      return (MessageStream<M>) outStream;
+    }
+    return null;
+  }
+
+  /**
+   * Method to create intermediate topics for {@link MessageStreamImpl#partitionBy(Function)} method.
+   *
+   * @param parKeyFn  the function to extract the partition key from the input message
+   * @param <PK>  the type of partition key
+   * @param <M>  the type of input message
+   * @return  the {@link OutputStream} object for the re-partitioned stream
+   */
+  <PK, M> MessageStreamImpl<M> createIntStream(Function<M, PK> parKeyFn) {
+    // TODO: placeholder to auto-generate intermediate streams via {@link StreamSpec}
+    StreamSpec streamSpec = new StreamSpec() {
+      @Override
+      public SystemStream getSystemStream() {
+        // TODO: should auto-generate intermedaite stream name here
+        return new SystemStream("intermediate", String.format("par-%d", StreamGraphImpl.this.opId));
+      }
+
+      @Override
+      public Properties getProperties() {
+        return null;
+      }
+    };
+
+    if (!this.inStreams.containsKey(streamSpec.getSystemStream())) {
+      this.inStreams.putIfAbsent(streamSpec.getSystemStream(), new IntermediateStreamImpl(this, streamSpec, null, null, parKeyFn));
+    }
+    IntermediateStreamImpl intStream = (IntermediateStreamImpl) this.inStreams.get(streamSpec.getSystemStream());
+    if (!this.outStreams.containsKey(streamSpec.getSystemStream())) {
+      this.outStreams.putIfAbsent(streamSpec.getSystemStream(), intStream);
+    }
+    return intStream;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/operators/StreamOperatorAdaptorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/StreamOperatorAdaptorTask.java b/samza-operator/src/main/java/org/apache/samza/operators/StreamOperatorAdaptorTask.java
deleted file mode 100644
index 152cd92..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/StreamOperatorAdaptorTask.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.data.IncomingSystemMessageEnvelope;
-import org.apache.samza.operators.data.MessageEnvelope;
-import org.apache.samza.operators.impl.OperatorImpl;
-import org.apache.samza.operators.impl.OperatorImpls;
-import org.apache.samza.system.IncomingMessageEnvelope;
-import org.apache.samza.system.SystemStreamPartition;
-import org.apache.samza.task.InitableTask;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.StreamTask;
-import org.apache.samza.task.TaskContext;
-import org.apache.samza.task.TaskCoordinator;
-import org.apache.samza.task.WindowableTask;
-
-import java.util.HashMap;
-import java.util.Map;
-
-
-/**
- * An {@link StreamTask} implementation that receives {@link IncomingSystemMessageEnvelope}s and propagates them
- * through the user's stream transformations defined in {@link StreamOperatorTask#transform(Map)} using the
- * {@link MessageStream} APIs.
- * <p>
- * This class brings all the operator API implementation components together and feeds the
- * {@link IncomingSystemMessageEnvelope}s into the transformation chains.
- * <p>
- * It accepts an instance of the user implemented {@link StreamOperatorTask}. When its own {@link #init(Config, TaskContext)}
- * method is called during startup, it creates a {@link MessageStreamImpl} corresponding to each of its input
- * {@link SystemStreamPartition}s and then calls the user's {@link StreamOperatorTask#transform(Map)} method.
- * <p>
- * When users invoke the methods on the {@link MessageStream} API to describe their stream transformations in the
- * {@link StreamOperatorTask#transform(Map)} method, the underlying {@link MessageStreamImpl} creates the
- * corresponding {@link org.apache.samza.operators.spec.OperatorSpec} to record information about the desired
- * transformation, and returns the output {@link MessageStream} to allow further transform chaining.
- * <p>
- * Once the user's transformation DAGs have been described for all {@link MessageStream}s (i.e., when the
- * {@link StreamOperatorTask#transform(Map)} call returns), it calls
- * {@link OperatorImpls#createOperatorImpls(MessageStreamImpl, TaskContext)} for each of the input
- * {@link MessageStreamImpl}. This instantiates the {@link org.apache.samza.operators.impl.OperatorImpl} DAG
- * corresponding to the aforementioned {@link org.apache.samza.operators.spec.OperatorSpec} DAG and returns the
- * root node of the DAG, which this class saves.
- * <p>
- * Now that it has the root for the DAG corresponding to each {@link SystemStreamPartition}, it can pass the message
- * envelopes received in {@link StreamTask#process(IncomingMessageEnvelope, MessageCollector, TaskCoordinator)} along
- * to the appropriate root nodes. From then on, each {@link org.apache.samza.operators.impl.OperatorImpl} propagates
- * its transformed output to the next set of {@link org.apache.samza.operators.impl.OperatorImpl}s.
- */
-public final class StreamOperatorAdaptorTask implements StreamTask, InitableTask, WindowableTask {
-
-  /**
-   * A mapping from each {@link SystemStreamPartition} to the root node of its operator chain DAG.
-   */
-  private final Map<SystemStreamPartition, OperatorImpl<IncomingSystemMessageEnvelope, ? extends MessageEnvelope>> operatorChains = new HashMap<>();
-
-  private final StreamOperatorTask userTask;
-
-  public StreamOperatorAdaptorTask(StreamOperatorTask userTask) {
-    this.userTask = userTask;
-  }
-
-  @Override
-  public final void init(Config config, TaskContext context) throws Exception {
-    if (this.userTask instanceof InitableTask) {
-      ((InitableTask) this.userTask).init(config, context);
-    }
-    Map<SystemStreamPartition, MessageStream<IncomingSystemMessageEnvelope>> messageStreams = new HashMap<>();
-    context.getSystemStreamPartitions().forEach(ssp -> messageStreams.put(ssp, new MessageStreamImpl<>()));
-    this.userTask.transform(messageStreams);
-    messageStreams.forEach((ssp, ms) ->
-        operatorChains.put(ssp, OperatorImpls.createOperatorImpls((MessageStreamImpl<IncomingSystemMessageEnvelope>) ms, context)));
-  }
-
-  @Override
-  public final void process(IncomingMessageEnvelope ime, MessageCollector collector, TaskCoordinator coordinator) {
-    this.operatorChains.get(ime.getSystemStreamPartition())
-        .onNext(new IncomingSystemMessageEnvelope(ime), collector, coordinator);
-  }
-
-  @Override
-  public final void window(MessageCollector collector, TaskCoordinator coordinator) throws Exception {
-    if (this.userTask instanceof WindowableTask) {
-      ((WindowableTask) this.userTask).window(collector, coordinator);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java b/samza-operator/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java
new file mode 100644
index 0000000..3583b92
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.functions;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.config.Config;
+import org.apache.samza.task.TaskContext;
+
+
+/**
+ * This defines the interface function a two-way join functions that takes input messages from two input
+ * {@link org.apache.samza.operators.MessageStream}s and merge them into a single output joined message in the join output
+ */
+@InterfaceStability.Unstable
+public interface PartialJoinFunction<K, M, OM, RM> extends InitFunction {
+
+  /**
+   * Method to perform join method on the two input messages
+   *
+   * @param m1  message from the first input stream
+   * @param om  message from the second input stream
+   * @return  the joined message in the output stream
+   */
+  RM apply(M m1, OM om);
+
+  /**
+   * Method to get the key from the input message
+   *
+   * @param message  the input message from the first strean
+   * @return  the join key in the {@code message}
+   */
+  K getKey(M message);
+
+  /**
+   * Method to get the key from the input message in the other stream
+   *
+   * @param message  the input message from the other stream
+   * @return  the join key in the {@code message}
+   */
+  K getOtherKey(OM message);
+
+  /**
+   * Init method to initialize the context for this {@link PartialJoinFunction}. The default implementation is NO-OP.
+   *
+   * @param config  the {@link Config} object for this task
+   * @param context  the {@link TaskContext} object for this task
+   */
+  default void init(Config config, TaskContext context) { }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorGraph.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorGraph.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorGraph.java
new file mode 100644
index 0000000..66336f8
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorGraph.java
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.impl;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.spec.OperatorSpec;
+import org.apache.samza.operators.spec.PartialJoinOperatorSpec;
+import org.apache.samza.operators.spec.SinkOperatorSpec;
+import org.apache.samza.operators.spec.StreamOperatorSpec;
+import org.apache.samza.operators.spec.WindowOperatorSpec;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.task.TaskContext;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+
+/**
+ * Instantiates the DAG of {@link OperatorImpl}s corresponding to the {@link OperatorSpec}s for a
+ * {@link MessageStreamImpl}
+ */
+public class OperatorGraph {
+
+  /**
+   * A {@link Map} from {@link OperatorSpec} to {@link OperatorImpl}. This map registers all {@link OperatorImpl} in the DAG
+   * of {@link OperatorImpl} in a {@link org.apache.samza.container.TaskInstance}. Each {@link OperatorImpl} is created
+   * according to a single instance of {@link OperatorSpec}.
+   */
+  private final Map<OperatorSpec, OperatorImpl> operators = new HashMap<>();
+
+  /**
+   * This {@link Map} describes the DAG of {@link OperatorImpl} that are chained together to process the input messages.
+   */
+  private final Map<SystemStream, RootOperatorImpl> operatorGraph = new HashMap<>();
+
+  /**
+   * Initialize the whole DAG of {@link OperatorImpl}s, based on the input {@link MessageStreamImpl} from the {@link org.apache.samza.operators.StreamGraph}.
+   * This method will traverse each input {@link org.apache.samza.operators.MessageStream} in the {@code inputStreams} and
+   * instantiate the corresponding {@link OperatorImpl} chains that take the {@link org.apache.samza.operators.MessageStream} as input.
+   *
+   * @param inputStreams  the map of input {@link org.apache.samza.operators.MessageStream}s
+   * @param config  the {@link Config} required to instantiate operators
+   * @param context  the {@link TaskContext} required to instantiate operators
+   */
+  public void init(Map<SystemStream, MessageStreamImpl> inputStreams, Config config, TaskContext context) {
+    inputStreams.forEach((ss, mstream) -> this.operatorGraph.put(ss, this.createOperatorImpls(mstream, config, context)));
+  }
+
+  /**
+   * Method to get the corresponding {@link RootOperatorImpl}
+   *
+   * @param ss  input {@link SystemStream}
+   * @param <M>  the type of input message
+   * @return  the {@link OperatorImpl} that starts processing the input message
+   */
+  public <M> OperatorImpl<M, M> get(SystemStream ss) {
+    return this.operatorGraph.get(ss);
+  }
+
+  /**
+   * Traverses the DAG of {@link OperatorSpec}s starting from the provided {@link MessageStreamImpl},
+   * creates the corresponding DAG of {@link OperatorImpl}s, and returns its root {@link RootOperatorImpl} node.
+   *
+   * @param source  the input {@link MessageStreamImpl} to instantiate {@link OperatorImpl}s for
+   * @param <M>  the type of messagess in the {@code source} {@link MessageStreamImpl}
+   * @param config  the {@link Config} required to instantiate operators
+   * @param context  the {@link TaskContext} required to instantiate operators
+   * @return  root node for the {@link OperatorImpl} DAG
+   */
+  private <M> RootOperatorImpl<M> createOperatorImpls(MessageStreamImpl<M> source, Config config,
+      TaskContext context) {
+    // since the source message stream might have multiple operator specs registered on it,
+    // create a new root node as a single point of entry for the DAG.
+    RootOperatorImpl<M> rootOperator = new RootOperatorImpl<>();
+    // create the pipeline/topology starting from the source
+    source.getRegisteredOperatorSpecs().forEach(registeredOperator -> {
+        // pass in the source and context s.t. stateful stream operators can initialize their stores
+        OperatorImpl<M, ?> operatorImpl =
+            this.createAndRegisterOperatorImpl(registeredOperator, source, config, context);
+        rootOperator.registerNextOperator(operatorImpl);
+      });
+    return rootOperator;
+  }
+
+  /**
+   * Helper method to recursively traverse the {@link OperatorSpec} DAG and instantiate and link the corresponding
+   * {@link OperatorImpl}s.
+   *
+   * @param operatorSpec  the operatorSpec registered with the {@code source}
+   * @param source  the source {@link MessageStreamImpl}
+   * @param <M>  type of input message
+   * @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
+   */
+  private <M> OperatorImpl<M, ?> createAndRegisterOperatorImpl(OperatorSpec operatorSpec,
+      MessageStreamImpl<M> source, Config config, TaskContext context) {
+    if (!operators.containsKey(operatorSpec)) {
+      OperatorImpl<M, ?> operatorImpl = createOperatorImpl(source, operatorSpec, config, context);
+      if (operators.putIfAbsent(operatorSpec, operatorImpl) == null) {
+        // this is the first time we've added the operatorImpl corresponding to the operatorSpec,
+        // so traverse and initialize and register the rest of the DAG.
+        // initialize the corresponding operator function
+        operatorSpec.init(config, context);
+        MessageStreamImpl nextStream = operatorSpec.getNextStream();
+        if (nextStream != null) {
+          Collection<OperatorSpec> registeredSpecs = nextStream.getRegisteredOperatorSpecs();
+          registeredSpecs.forEach(registeredSpec -> {
+              OperatorImpl subImpl = this.createAndRegisterOperatorImpl(registeredSpec, nextStream, config, context);
+              operatorImpl.registerNextOperator(subImpl);
+            });
+        }
+        return operatorImpl;
+      }
+    }
+
+    // the implementation corresponding to operatorSpec has already been instantiated
+    // and registered, so we do not need to traverse the DAG further.
+    return operators.get(operatorSpec);
+  }
+
+  /**
+   * Creates a new {@link OperatorImpl} instance for the provided {@link OperatorSpec}.
+   *
+   * @param source  the source {@link MessageStreamImpl}
+   * @param <M>  type of input message
+   * @param operatorSpec  the immutable {@link OperatorSpec} definition.
+   * @param config  the {@link Config} required to instantiate operators
+   * @param context  the {@link TaskContext} required to instantiate operators
+   * @return  the {@link OperatorImpl} implementation instance
+   */
+  private static <M> OperatorImpl<M, ?> createOperatorImpl(MessageStreamImpl<M> source, OperatorSpec operatorSpec, Config config, TaskContext context) {
+    if (operatorSpec instanceof StreamOperatorSpec) {
+      StreamOperatorSpec<M, ?> streamOpSpec = (StreamOperatorSpec<M, ?>) operatorSpec;
+      return new StreamOperatorImpl<>(streamOpSpec, source, config, context);
+    } else if (operatorSpec instanceof SinkOperatorSpec) {
+      return new SinkOperatorImpl<>((SinkOperatorSpec<M>) operatorSpec, config, context);
+    } else if (operatorSpec instanceof WindowOperatorSpec) {
+      return new WindowOperatorImpl<>((WindowOperatorSpec<M, ?, ?>) operatorSpec, source, config, context);
+    } else if (operatorSpec instanceof PartialJoinOperatorSpec) {
+      return new PartialJoinOperatorImpl<>((PartialJoinOperatorSpec) operatorSpec, source, config, context);
+    }
+    throw new IllegalArgumentException(
+        String.format("Unsupported OperatorSpec: %s", operatorSpec.getClass().getName()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
index c77914e..abb1fa9 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
@@ -18,10 +18,7 @@
  */
 package org.apache.samza.operators.impl;
 
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
 
 import java.util.HashSet;
@@ -31,32 +28,24 @@ import java.util.Set;
 /**
  * Abstract base class for all stream operator implementations.
  */
-public abstract class OperatorImpl<M extends MessageEnvelope, RM extends MessageEnvelope> {
+public abstract class OperatorImpl<M, RM> {
 
-  private final Set<OperatorImpl<RM, ? extends MessageEnvelope>> nextOperators = new HashSet<>();
+  private final Set<OperatorImpl<RM, ?>> nextOperators = new HashSet<>();
 
   /**
    * Register the next operator in the chain that this operator should propagate its output to.
    * @param nextOperator  the next operator in the chain.
    */
-  void registerNextOperator(OperatorImpl<RM, ? extends MessageEnvelope> nextOperator) {
+  void registerNextOperator(OperatorImpl<RM, ?> nextOperator) {
     nextOperators.add(nextOperator);
   }
 
   /**
-   * Initialize the initial state for stateful operators.
-   *
-   * @param source  the source that this {@link OperatorImpl} operator is registered with
-   * @param context  the task context to initialize the operator implementation
-   */
-  public void init(MessageStream<M> source, TaskContext context) {}
-
-  /**
    * Perform the transformation required for this operator and call the downstream operators.
    *
    * Must call {@link #propagateResult} to propage the output to registered downstream operators correctly.
    *
-   * @param message  the input {@link MessageEnvelope}
+   * @param message  the input message
    * @param collector  the {@link MessageCollector} in the context
    * @param coordinator  the {@link TaskCoordinator} in the context
    */
@@ -67,11 +56,12 @@ public abstract class OperatorImpl<M extends MessageEnvelope, RM extends Message
    *
    * This method <b>must</b> be called from {@link #onNext} to propagate the operator output correctly.
    *
-   * @param outputMessage  output {@link MessageEnvelope}
+   * @param outputMessage  output message
    * @param collector  the {@link MessageCollector} in the context
    * @param coordinator  the {@link TaskCoordinator} in the context
    */
   void propagateResult(RM outputMessage, MessageCollector collector, TaskCoordinator coordinator) {
     nextOperators.forEach(sub -> sub.onNext(outputMessage, collector, coordinator));
   }
+
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java
deleted file mode 100644
index 02095cb..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators.impl;
-
-
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.MessageStreamImpl;
-import org.apache.samza.operators.data.MessageEnvelope;
-import org.apache.samza.operators.spec.OperatorSpec;
-import org.apache.samza.operators.spec.PartialJoinOperatorSpec;
-import org.apache.samza.operators.spec.SinkOperatorSpec;
-import org.apache.samza.operators.spec.StreamOperatorSpec;
-import org.apache.samza.operators.spec.WindowOperatorSpec;
-import org.apache.samza.operators.windows.WindowPane;
-import org.apache.samza.task.TaskContext;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-
-/**
- * Instantiates the DAG of {@link OperatorImpl}s corresponding to the {@link OperatorSpec}s for a
- * {@link MessageStreamImpl}
- */
-public class OperatorImpls {
-
-  /**
-   * Holds the mapping between the {@link OperatorSpec} and {@link OperatorImpl}s instances.
-   */
-  private static final Map<OperatorSpec, OperatorImpl> OPERATOR_IMPLS = new ConcurrentHashMap<>();
-
-  /**
-   * Traverses the DAG of {@link OperatorSpec}s starting from the provided {@link MessageStreamImpl},
-   * creates the corresponding DAG of {@link OperatorImpl}s, and returns its root {@link RootOperatorImpl} node.
-   *
-   * @param source  the input {@link MessageStreamImpl} to instantiate {@link OperatorImpl}s for
-   * @param <M>  the type of {@link MessageEnvelope}s in the {@code source} {@link MessageStream}
-   * @param context  the {@link TaskContext} required to instantiate operators
-   * @return  root node for the {@link OperatorImpl} DAG
-   */
-  public static <M extends MessageEnvelope> RootOperatorImpl createOperatorImpls(MessageStreamImpl<M> source, TaskContext context) {
-    // since the source message stream might have multiple operator specs registered on it,
-    // create a new root node as a single point of entry for the DAG.
-    RootOperatorImpl<M> rootOperator = new RootOperatorImpl<>();
-    // create the pipeline/topology starting from the source
-    source.getRegisteredOperatorSpecs().forEach(registeredOperator -> {
-        // pass in the source and context s.t. stateful stream operators can initialize their stores
-        OperatorImpl<M, ? extends MessageEnvelope> operatorImpl =
-            createAndRegisterOperatorImpl(registeredOperator, source, context);
-        rootOperator.registerNextOperator(operatorImpl);
-      });
-    return rootOperator;
-  }
-
-  /**
-   * Helper method to recursively traverse the {@link OperatorSpec} DAG and instantiate and link the corresponding
-   * {@link OperatorImpl}s.
-   *
-   * @param operatorSpec  the operatorSpec registered with the {@code source}
-   * @param source  the source {@link MessageStreamImpl}
-   * @param context  the context of the task
-   * @return  the operator implementation for the operatorSpec
-   */
-  private static <M extends MessageEnvelope> OperatorImpl<M, ? extends MessageEnvelope> createAndRegisterOperatorImpl(OperatorSpec operatorSpec,
-      MessageStream source, TaskContext context) {
-    if (!OPERATOR_IMPLS.containsKey(operatorSpec)) {
-      OperatorImpl<M, ? extends MessageEnvelope> operatorImpl = createOperatorImpl(operatorSpec);
-      if (OPERATOR_IMPLS.putIfAbsent(operatorSpec, operatorImpl) == null) {
-        // this is the first time we've added the operatorImpl corresponding to the operatorSpec,
-        // so traverse and initialize and register the rest of the DAG.
-        MessageStream<? extends MessageEnvelope> outStream = operatorSpec.getOutputStream();
-        Collection<OperatorSpec> registeredSpecs = ((MessageStreamImpl) outStream).getRegisteredOperatorSpecs();
-        registeredSpecs.forEach(registeredSpec -> {
-            OperatorImpl subImpl = createAndRegisterOperatorImpl(registeredSpec, outStream, context);
-            operatorImpl.registerNextOperator(subImpl);
-          });
-        operatorImpl.init(source, context);
-        return operatorImpl;
-      }
-    }
-
-    // the implementation corresponding to operatorSpec has already been instantiated
-    // and registered, so we do not need to traverse the DAG further.
-    return OPERATOR_IMPLS.get(operatorSpec);
-  }
-
-  /**
-   * Creates a new {@link OperatorImpl} instance for the provided {@link OperatorSpec}.
-   *
-   * @param operatorSpec  the immutable {@link OperatorSpec} definition.
-   * @param <M>  type of input {@link MessageEnvelope}
-   * @return  the {@link OperatorImpl} implementation instance
-   */
-  protected static <M extends MessageEnvelope> OperatorImpl<M, ? extends MessageEnvelope> createOperatorImpl(OperatorSpec operatorSpec) {
-    if (operatorSpec instanceof StreamOperatorSpec) {
-      return new StreamOperatorImpl<>((StreamOperatorSpec<M, ? extends MessageEnvelope>) operatorSpec);
-    } else if (operatorSpec instanceof SinkOperatorSpec) {
-      return new SinkOperatorImpl<>((SinkOperatorSpec<M>) operatorSpec);
-    } else if (operatorSpec instanceof WindowOperatorSpec) {
-      return new WindowOperatorImpl<>((WindowOperatorSpec<M, ?, ?, ?, ? extends WindowPane>) operatorSpec);
-    } else if (operatorSpec instanceof PartialJoinOperatorSpec) {
-      return new PartialJoinOperatorImpl<>((PartialJoinOperatorSpec) operatorSpec);
-    }
-    throw new IllegalArgumentException(
-        String.format("Unsupported OperatorSpec: %s", operatorSpec.getClass().getName()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java
index 90569b4..c8515e1 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java
@@ -18,9 +18,11 @@
  */
 package org.apache.samza.operators.impl;
 
-import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
 import org.apache.samza.operators.spec.PartialJoinOperatorSpec;
 import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
 
 
@@ -28,14 +30,13 @@ import org.apache.samza.task.TaskCoordinator;
  * Implementation of a {@link PartialJoinOperatorSpec}. This class implements function
  * that only takes in one input stream among all inputs to the join and generate the join output.
  *
- * @param <M>  type of {@link MessageEnvelope}s in the input stream
- * @param <JM>  type of {@link MessageEnvelope}s in the stream to join with
- * @param <RM>  type of {@link MessageEnvelope}s in the joined stream
+ * @param <M>  type of messages in the input stream
+ * @param <JM>  type of messages in the stream to join with
+ * @param <RM>  type of messages in the joined stream
  */
-class PartialJoinOperatorImpl<M extends MessageEnvelope<K, ?>, K, JM extends MessageEnvelope<K, ?>, RM extends MessageEnvelope>
-    extends OperatorImpl<M, RM> {
+class PartialJoinOperatorImpl<M, K, JM, RM> extends OperatorImpl<M, RM> {
 
-  PartialJoinOperatorImpl(PartialJoinOperatorSpec<M, K, JM, RM> joinOp) {
+  PartialJoinOperatorImpl(PartialJoinOperatorSpec<M, K, JM, RM> joinOp, MessageStreamImpl<M> source, Config config, TaskContext context) {
     // TODO: implement PartialJoinOperatorImpl constructor
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.java
index 7132b86..4b30a5d 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.java
@@ -18,16 +18,15 @@
  */
 package org.apache.samza.operators.impl;
 
-import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.task.MessageCollector;
 import org.apache.samza.task.TaskCoordinator;
 
 
 /**
- * A no-op operator implementation that forwards incoming {@link MessageEnvelope}s to all of its subscribers.
- * @param <M>  type of incoming {@link MessageEnvelope}s
+ * A no-op operator implementation that forwards incoming messages to all of its subscribers.
+ * @param <M>  type of incoming messages
  */
-final class RootOperatorImpl<M extends MessageEnvelope> extends OperatorImpl<M, M> {
+final class RootOperatorImpl<M> extends OperatorImpl<M, M> {
 
   @Override
   public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {

http://git-wip-us.apache.org/repos/asf/samza/blob/b3dd886d/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java
new file mode 100644
index 0000000..2bb362c
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.impl;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.spec.WindowOperatorSpec;
+import org.apache.samza.operators.windows.WindowPane;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+
+
+/**
+ * Default implementation class of a {@link WindowOperatorSpec} for a session window.
+ *
+ * @param <M>  the type of input message
+ * @param <RK>  the type of window key
+ * @param <WV>  the type of window state
+ */
+class SessionWindowOperatorImpl<M, RK, WV> extends OperatorImpl<M, WindowPane<RK, WV>> {
+
+  private final WindowOperatorSpec<M, RK, WV> windowSpec;
+
+  SessionWindowOperatorImpl(WindowOperatorSpec<M, RK, WV> windowSpec, MessageStreamImpl<M> source, Config config, TaskContext context) {
+    this.windowSpec = windowSpec;
+  }
+
+  @Override
+  public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
+  }
+
+  public void onTimer(MessageCollector collector, TaskCoordinator coordinator) {
+    // This is to periodically check the timeout triggers to get the list of window states to be updated
+  }
+}


[10/14] samza git commit: SAMZA-1073: top-level fluent API

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
index abed03f..41d1778 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
@@ -18,21 +18,22 @@
  */
 package org.apache.samza.operators.impl;
 
-import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.config.Config;
 import org.apache.samza.operators.functions.SinkFunction;
 import org.apache.samza.operators.spec.SinkOperatorSpec;
 import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
 
 
 /**
  * Implementation for {@link SinkOperatorSpec}
  */
-class SinkOperatorImpl<M extends MessageEnvelope> extends OperatorImpl<M, MessageEnvelope> {
+class SinkOperatorImpl<M> extends OperatorImpl<M, M> {
 
   private final SinkFunction<M> sinkFn;
 
-  SinkOperatorImpl(SinkOperatorSpec<M> sinkOp) {
+  SinkOperatorImpl(SinkOperatorSpec<M> sinkOp, Config config, TaskContext context) {
     this.sinkFn = sinkOp.getSinkFn();
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java
index 3a5c56e..644de20 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java
@@ -18,24 +18,26 @@
  */
 package org.apache.samza.operators.impl;
 
-import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
 import org.apache.samza.operators.functions.FlatMapFunction;
 import org.apache.samza.operators.spec.StreamOperatorSpec;
 import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
 
 
 /**
- * A StreamOperator that accepts a 1:n transform function and applies it to each incoming {@link MessageEnvelope}.
+ * A StreamOperator that accepts a 1:n transform function and applies it to each incoming message.
  *
- * @param <M>  type of {@link MessageEnvelope} in the input stream
- * @param <RM>  type of {@link MessageEnvelope} in the output stream
+ * @param <M>  type of message in the input stream
+ * @param <RM>  type of message in the output stream
  */
-class StreamOperatorImpl<M extends MessageEnvelope, RM extends MessageEnvelope> extends OperatorImpl<M, RM> {
+class StreamOperatorImpl<M, RM> extends OperatorImpl<M, RM> {
 
   private final FlatMapFunction<M, RM> transformFn;
 
-  StreamOperatorImpl(StreamOperatorSpec<M, RM> streamOperatorSpec) {
+  StreamOperatorImpl(StreamOperatorSpec<M, RM> streamOperatorSpec, MessageStreamImpl<M> source, Config config, TaskContext context) {
     this.transformFn = streamOperatorSpec.getTransformFn();
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java
index a5b71a7..af00553 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java
@@ -18,18 +18,21 @@
  */
 package org.apache.samza.operators.impl;
 
-import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
 import org.apache.samza.operators.spec.WindowOperatorSpec;
 import org.apache.samza.operators.windows.WindowPane;
 import org.apache.samza.operators.windows.internal.WindowInternal;
 import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
 import org.apache.samza.task.TaskCoordinator;
 
-public class WindowOperatorImpl<M extends MessageEnvelope, K, WK, WV, WM extends WindowPane<WK, WV>> extends OperatorImpl<M, WM> {
+public class WindowOperatorImpl<M, WK, WV> extends OperatorImpl<M, WindowPane<WK, WV>> {
 
-  private final WindowInternal<M, K, WV> window;
+  private final WindowInternal<M, WK, WV> window;
 
-  public WindowOperatorImpl(WindowOperatorSpec spec) {
+  public WindowOperatorImpl(WindowOperatorSpec spec, MessageStreamImpl<M> source, Config config, TaskContext context) {
+    // source, config, and context are used to initialize the window kv-store
     window = spec.getWindow();
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
index 8b75cdc..1444662 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
@@ -18,20 +18,45 @@
  */
 package org.apache.samza.operators.spec;
 
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.task.TaskContext;
 
 
 /**
  * A stateless serializable stream operator specification that holds all the information required
- * to transform the input {@link MessageStream} and produce the output {@link MessageStream}.
+ * to transform the input {@link MessageStreamImpl} and produce the output {@link MessageStreamImpl}.
+ *
+ * @param <OM>  the type of output message from the operator
  */
-public interface OperatorSpec<OM extends MessageEnvelope> {
+@InterfaceStability.Unstable
+public interface OperatorSpec<OM> {
+
+  enum OpCode {
+    MAP,
+    FLAT_MAP,
+    FILTER,
+    SINK,
+    SEND_TO,
+    JOIN,
+    WINDOW,
+    MERGE,
+    PARTITION_BY
+  }
+
 
   /**
-   * Get the output stream containing transformed {@link MessageEnvelope} produced by this operator.
-   * @return  the output stream containing transformed {@link MessageEnvelope} produced by this operator.
+   * Get the output stream containing transformed messages produced by this operator.
+   * @return  the output stream containing transformed messages produced by this operator.
    */
-  MessageStream<OM> getOutputStream();
+  MessageStreamImpl<OM> getNextStream();
 
+  /**
+   * Init method to initialize the context for this {@link OperatorSpec}. The default implementation is NO-OP.
+   *
+   * @param config  the {@link Config} object for this task
+   * @param context  the {@link TaskContext} object for this task
+   */
+  default void init(Config config, TaskContext context) { }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java
index fc25929..d626852 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java
@@ -19,16 +19,21 @@
 
 package org.apache.samza.operators.spec;
 
-import org.apache.samza.operators.data.MessageEnvelope;
+import java.util.Collection;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.OutputStream;
+import org.apache.samza.operators.StreamGraphImpl;
+import org.apache.samza.operators.functions.FilterFunction;
 import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.functions.MapFunction;
+import org.apache.samza.operators.functions.PartialJoinFunction;
 import org.apache.samza.operators.functions.SinkFunction;
-import org.apache.samza.operators.MessageStreamImpl;
 import org.apache.samza.operators.windows.WindowPane;
 import org.apache.samza.operators.windows.internal.WindowInternal;
 
 import java.util.ArrayList;
-import java.util.UUID;
-import java.util.function.BiFunction;
+import org.apache.samza.task.TaskContext;
 
 
 /**
@@ -38,80 +43,168 @@ public class OperatorSpecs {
 
   private OperatorSpecs() {}
 
-  private static String getOperatorId() {
-    // TODO: need to change the IDs to be a consistent, durable IDs that can be recovered across container and job restarts
-    return UUID.randomUUID().toString();
+  /**
+   * Creates a {@link StreamOperatorSpec} for {@link MapFunction}
+   *
+   * @param mapFn  the map function
+   * @param graph  the {@link StreamGraphImpl} object
+   * @param output  the output {@link MessageStreamImpl} object
+   * @param <M>  type of input message
+   * @param <OM>  type of output message
+   * @return  the {@link StreamOperatorSpec}
+   */
+  public static <M, OM> StreamOperatorSpec<M, OM> createMapOperatorSpec(MapFunction<M, OM> mapFn, StreamGraphImpl graph, MessageStreamImpl<OM> output) {
+    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);
+      }
+    }, output, OperatorSpec.OpCode.MAP, graph.getNextOpId());
+  }
+
+  /**
+   * Creates a {@link StreamOperatorSpec} for {@link FilterFunction}
+   *
+   * @param filterFn  the transformation function
+   * @param graph  the {@link StreamGraphImpl} object
+   * @param output  the output {@link MessageStreamImpl} object
+   * @param <M>  type of input message
+   * @return  the {@link StreamOperatorSpec}
+   */
+  public static <M> StreamOperatorSpec<M, M> createFilterOperatorSpec(FilterFunction<M> filterFn, StreamGraphImpl graph, MessageStreamImpl<M> output) {
+    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);
+      }
+    }, output, OperatorSpec.OpCode.FILTER, graph.getNextOpId());
   }
 
   /**
    * Creates a {@link StreamOperatorSpec}.
    *
    * @param transformFn  the transformation function
-   * @param <M>  type of input {@link MessageEnvelope}
-   * @param <OM>  type of output {@link MessageEnvelope}
+   * @param graph  the {@link StreamGraphImpl} object
+   * @param output  the output {@link MessageStreamImpl} object
+   * @param <M>  type of input message
+   * @param <OM>  type of output message
    * @return  the {@link StreamOperatorSpec}
    */
-  public static <M extends MessageEnvelope, OM extends MessageEnvelope> StreamOperatorSpec<M, OM> createStreamOperatorSpec(
-      FlatMapFunction<M, OM> transformFn) {
-    return new StreamOperatorSpec<>(transformFn);
+  public static <M, OM> StreamOperatorSpec<M, OM> createStreamOperatorSpec(
+      FlatMapFunction<M, OM> transformFn, StreamGraphImpl graph, MessageStreamImpl<OM> output) {
+    return new StreamOperatorSpec<>(transformFn, output, OperatorSpec.OpCode.FLAT_MAP, graph.getNextOpId());
+  }
+
+  /**
+   * Creates a {@link SinkOperatorSpec}.
+   *
+   * @param sinkFn  the sink function
+   * @param <M>  type of input message
+   * @param graph  the {@link StreamGraphImpl} object
+   * @return  the {@link SinkOperatorSpec}
+   */
+  public static <M> SinkOperatorSpec<M> createSinkOperatorSpec(SinkFunction<M> sinkFn, StreamGraphImpl graph) {
+    return new SinkOperatorSpec<>(sinkFn, OperatorSpec.OpCode.SINK, graph.getNextOpId());
+  }
+
+  /**
+   * Creates a {@link SinkOperatorSpec}.
+   *
+   * @param sinkFn  the sink function
+   * @param graph  the {@link StreamGraphImpl} object
+   * @param stream  the {@link OutputStream} where the message is sent to
+   * @param <M>  type of input message
+   * @return  the {@link SinkOperatorSpec}
+   */
+  public static <M> SinkOperatorSpec<M> createSendToOperatorSpec(SinkFunction<M> sinkFn, StreamGraphImpl graph, OutputStream<M> stream) {
+    return new SinkOperatorSpec<>(sinkFn, OperatorSpec.OpCode.SEND_TO, graph.getNextOpId(), stream);
   }
 
   /**
    * Creates a {@link SinkOperatorSpec}.
    *
    * @param sinkFn  the sink function
-   * @param <M>  type of input {@link MessageEnvelope}
+   * @param graph  the {@link StreamGraphImpl} object
+   * @param stream  the {@link OutputStream} where the message is sent to
+   * @param <M>  type of input message
    * @return  the {@link SinkOperatorSpec}
    */
-  public static <M extends MessageEnvelope> SinkOperatorSpec<M> createSinkOperatorSpec(SinkFunction<M> sinkFn) {
-    return new SinkOperatorSpec<>(sinkFn);
+  public static <M> SinkOperatorSpec<M> createPartitionOperatorSpec(SinkFunction<M> sinkFn, StreamGraphImpl graph, OutputStream<M> stream) {
+    return new SinkOperatorSpec<>(sinkFn, OperatorSpec.OpCode.PARTITION_BY, graph.getNextOpId(), stream);
   }
 
   /**
    * Creates a {@link WindowOperatorSpec}.
    *
    * @param window the description of the window.
-   * @param <M> the type of input {@link MessageEnvelope}
-   * @param <K> the type of key in the {@link MessageEnvelope} in this {@link org.apache.samza.operators.MessageStream}. If a key is specified,
-   *            results are emitted per-key
+   * @param graph  the {@link StreamGraphImpl} object
+   * @param wndOutput  the window output {@link MessageStreamImpl} object
+   * @param <M> the type of input message
    * @param <WK> the type of key in the {@link WindowPane}
    * @param <WV> the type of value in the window
-   * @param <WM> the type of output {@link WindowPane}
    * @return  the {@link WindowOperatorSpec}
    */
 
-  public static <M extends MessageEnvelope, K, WK, WV, WM extends WindowPane<WK, WV>> WindowOperatorSpec<M, K, WK, WV, WM> createWindowOperatorSpec(WindowInternal<M, K, WV> window) {
-    return new WindowOperatorSpec<>(window, OperatorSpecs.getOperatorId());
+  public static <M, WK, WV> WindowOperatorSpec<M, WK, WV> createWindowOperatorSpec(
+      WindowInternal<M, WK, WV> window, StreamGraphImpl graph, MessageStreamImpl<WindowPane<WK, WV>> wndOutput) {
+    return new WindowOperatorSpec<>(window, wndOutput, graph.getNextOpId());
   }
 
   /**
    * Creates a {@link PartialJoinOperatorSpec}.
    *
    * @param partialJoinFn  the join function
+   * @param graph  the {@link StreamGraphImpl} object
    * @param joinOutput  the output {@link MessageStreamImpl}
-   * @param <M>  type of input {@link MessageEnvelope}
+   * @param <M>  type of input message
    * @param <K>  type of join key
-   * @param <JM>  the type of {@link MessageEnvelope} in the other join stream
-   * @param <OM>  the type of {@link MessageEnvelope} in the join output
+   * @param <JM>  the type of message in the other join stream
+   * @param <OM>  the type of message in the join output
    * @return  the {@link PartialJoinOperatorSpec}
    */
-  public static <M extends MessageEnvelope<K, ?>, K, JM extends MessageEnvelope<K, ?>, OM extends MessageEnvelope> PartialJoinOperatorSpec<M, K, JM, OM> createPartialJoinOperatorSpec(
-      BiFunction<M, JM, OM> partialJoinFn, MessageStreamImpl<OM> joinOutput) {
-    return new PartialJoinOperatorSpec<>(partialJoinFn, joinOutput, OperatorSpecs.getOperatorId());
+  public static <M, K, JM, OM> PartialJoinOperatorSpec<M, K, JM, OM> createPartialJoinOperatorSpec(
+      PartialJoinFunction<K, M, JM, OM> partialJoinFn, StreamGraphImpl graph, MessageStreamImpl<OM> joinOutput) {
+    return new PartialJoinOperatorSpec<>(partialJoinFn, joinOutput, graph.getNextOpId());
   }
 
   /**
    * Creates a {@link StreamOperatorSpec} with a merger function.
    *
+   * @param graph  the {@link StreamGraphImpl} object
    * @param mergeOutput  the output {@link MessageStreamImpl} from the merger
-   * @param <M>  the type of input {@link MessageEnvelope}
+   * @param <M>  the type of input message
    * @return  the {@link StreamOperatorSpec} for the merge
    */
-  public static <M extends MessageEnvelope> StreamOperatorSpec<M, M> createMergeOperatorSpec(MessageStreamImpl<M> mergeOutput) {
-    return new StreamOperatorSpec<M, M>(t ->
-      new ArrayList<M>() { {
-          this.add(t);
-        } },
-      mergeOutput);
+  public static <M> StreamOperatorSpec<M, M> createMergeOperatorSpec(StreamGraphImpl graph, MessageStreamImpl<M> mergeOutput) {
+    return new StreamOperatorSpec<M, M>(message ->
+        new ArrayList<M>() {
+          {
+            this.add(message);
+          }
+        },
+        mergeOutput, OperatorSpec.OpCode.MERGE, graph.getNextOpId());
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java
index e6d77f6..e057c2b 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java
@@ -18,63 +18,69 @@
  */
 package org.apache.samza.operators.spec;
 
-import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.config.Config;
 import org.apache.samza.operators.MessageStreamImpl;
-
-import java.util.function.BiFunction;
+import org.apache.samza.operators.functions.PartialJoinFunction;
+import org.apache.samza.task.TaskContext;
 
 
 /**
- * Spec for the partial join operator that takes {@link MessageEnvelope}s from one input stream, joins with buffered
- * {@link MessageEnvelope}s from another stream, and produces join results to an output {@link MessageStreamImpl}.
+ * Spec for the partial join operator that takes messages from one input stream, joins with buffered
+ * messages from another stream, and produces join results to an output {@link MessageStreamImpl}.
  *
- * @param <M>  the type of input {@link MessageEnvelope}
+ * @param <M>  the type of input message
  * @param <K>  the type of join key
- * @param <JM>  the type of {@link MessageEnvelope} in the other join stream
- * @param <RM>  the type of {@link MessageEnvelope} in the join output stream
+ * @param <JM>  the type of message in the other join stream
+ * @param <RM>  the type of message in the join output stream
  */
-public class PartialJoinOperatorSpec<M extends MessageEnvelope<K, ?>, K, JM extends MessageEnvelope<K, ?>, RM extends MessageEnvelope>
-    implements OperatorSpec<RM> {
+public class PartialJoinOperatorSpec<M, K, JM, RM> implements OperatorSpec<RM> {
 
   private final MessageStreamImpl<RM> joinOutput;
 
   /**
-   * The transformation function of {@link PartialJoinOperatorSpec} that takes an input {@link MessageEnvelope} of
-   * type {@code M}, joins with a stream of buffered {@link MessageEnvelope}s of type {@code JM} from another stream,
-   * and generates a joined result {@link MessageEnvelope} of type {@code RM}.
+   * The transformation function of {@link PartialJoinOperatorSpec} that takes an input message of
+   * type {@code M}, joins with a stream of buffered messages of type {@code JM} from another stream,
+   * and generates a joined result message of type {@code RM}.
    */
-  private final BiFunction<M, JM, RM> transformFn;
+  private final PartialJoinFunction<K, M, JM, RM> transformFn;
 
 
   /**
    * The unique ID for this operator.
    */
-  private final String operatorId;
+  private final int opId;
 
   /**
    * Default constructor for a {@link PartialJoinOperatorSpec}.
    *
-   * @param partialJoinFn  partial join function that take type {@code M} of input {@link MessageEnvelope} and join
-   *                       w/ type {@code JM} of buffered {@link MessageEnvelope} from another stream
+   * @param partialJoinFn  partial join function that take type {@code M} of input message and join
+   *                       w/ type {@code JM} of buffered message from another stream
    * @param joinOutput  the output {@link MessageStreamImpl} of the join results
    */
-  PartialJoinOperatorSpec(BiFunction<M, JM, RM> partialJoinFn, MessageStreamImpl<RM> joinOutput, String operatorId) {
+  PartialJoinOperatorSpec(PartialJoinFunction<K, M, JM, RM> partialJoinFn, MessageStreamImpl<RM> joinOutput, int opId) {
     this.joinOutput = joinOutput;
     this.transformFn = partialJoinFn;
-    this.operatorId = operatorId;
+    this.opId = opId;
   }
 
   @Override
-  public String toString() {
-    return this.operatorId;
-  }
-
-  @Override
-  public MessageStreamImpl<RM> getOutputStream() {
+  public MessageStreamImpl<RM> getNextStream() {
     return this.joinOutput;
   }
 
-  public BiFunction<M, JM, RM> getTransformFn() {
+  public PartialJoinFunction<K, M, JM, RM> getTransformFn() {
     return this.transformFn;
   }
+
+  public OperatorSpec.OpCode getOpCode() {
+    return OpCode.JOIN;
+  }
+
+  public int getOpId() {
+    return this.opId;
+  }
+
+  @Override public void init(Config config, TaskContext context) {
+    this.transformFn.init(config, context);
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java
index 4348bc0..ba30d67 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java
@@ -18,18 +18,30 @@
  */
 package org.apache.samza.operators.spec;
 
-import org.apache.samza.operators.data.MessageEnvelope;
-import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.config.Config;
 import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.OutputStream;
+import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.task.TaskContext;
 
 
 /**
  * The spec for a sink operator that accepts user-defined logic to output a {@link MessageStreamImpl} to an external
  * system. This is a terminal operator and does allows further operator chaining.
  *
- * @param <M>  the type of input {@link MessageEnvelope}
+ * @param <M>  the type of input message
  */
-public class SinkOperatorSpec<M extends MessageEnvelope> implements OperatorSpec {
+public class SinkOperatorSpec<M> implements OperatorSpec {
+
+  /**
+   * {@link OpCode} for this {@link SinkOperatorSpec}
+   */
+  private final OperatorSpec.OpCode opCode;
+
+  /**
+   * The unique ID for this operator.
+   */
+  private final int opId;
 
   /**
    * The user-defined sink function
@@ -37,14 +49,40 @@ public class SinkOperatorSpec<M extends MessageEnvelope> implements OperatorSpec
   private final SinkFunction<M> sinkFn;
 
   /**
-   * Default constructor for a {@link SinkOperatorSpec}.
+   * Potential output stream defined by the {@link SinkFunction}
+   */
+  private final OutputStream<M> outStream;
+
+  /**
+   * Default constructor for a {@link SinkOperatorSpec} w/o an output stream. (e.g. output is sent to remote database)
+   *
+   * @param sinkFn  a user defined {@link SinkFunction} that will be called with the output message,
+   *                the output {@link org.apache.samza.task.MessageCollector} and the
+   *                {@link org.apache.samza.task.TaskCoordinator}.
+   * @param opCode  the specific {@link OpCode} for this {@link SinkOperatorSpec}. It could be {@link OpCode#SINK}, {@link OpCode#SEND_TO},
+   *                or {@link OpCode#PARTITION_BY}
+   * @param opId  the unique id of this {@link SinkOperatorSpec} in the {@link org.apache.samza.operators.StreamGraph}
+   */
+  SinkOperatorSpec(SinkFunction<M> sinkFn, OperatorSpec.OpCode opCode, int opId) {
+    this(sinkFn, opCode, opId, null);
+  }
+
+  /**
+   * Default constructor for a {@link SinkOperatorSpec} that sends the output to an {@link OutputStream}
    *
-   * @param sinkFn  a user defined {@link SinkFunction} that will be called with the output {@link MessageEnvelope},
+   * @param sinkFn  a user defined {@link SinkFunction} that will be called with the output message,
    *                the output {@link org.apache.samza.task.MessageCollector} and the
    *                {@link org.apache.samza.task.TaskCoordinator}.
+   * @param opCode  the specific {@link OpCode} for this {@link SinkOperatorSpec}. It could be {@link OpCode#SINK}, {@link OpCode#SEND_TO},
+   *                or {@link OpCode#PARTITION_BY}
+   * @param opId  the unique id of this {@link SinkOperatorSpec} in the {@link org.apache.samza.operators.StreamGraph}
+   * @param opId  the {@link OutputStream} for this {@link SinkOperatorSpec}
    */
-  SinkOperatorSpec(SinkFunction<M> sinkFn) {
+  SinkOperatorSpec(SinkFunction<M> sinkFn, OperatorSpec.OpCode opCode, int opId, OutputStream<M> outStream) {
     this.sinkFn = sinkFn;
+    this.opCode = opCode;
+    this.opId = opId;
+    this.outStream = outStream;
   }
 
   /**
@@ -52,11 +90,27 @@ public class SinkOperatorSpec<M extends MessageEnvelope> implements OperatorSpec
    * @return  null
    */
   @Override
-  public MessageStreamImpl getOutputStream() {
+  public MessageStreamImpl<M> getNextStream() {
     return null;
   }
 
   public SinkFunction<M> getSinkFn() {
     return this.sinkFn;
   }
+
+  public OperatorSpec.OpCode getOpCode() {
+    return this.opCode;
+  }
+
+  public int getOpId() {
+    return this.opId;
+  }
+
+  public OutputStream<M> getOutStream() {
+    return this.outStream;
+  }
+
+  @Override public void init(Config config, TaskContext context) {
+    this.sinkFn.init(config, context);
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java
index ed18da4..d7813f7 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java
@@ -18,50 +18,74 @@
  */
 package org.apache.samza.operators.spec;
 
-import org.apache.samza.operators.data.MessageEnvelope;
-import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.config.Config;
 import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.task.TaskContext;
 
 
 /**
- * The spec for a linear stream operator that outputs 0 or more {@link MessageEnvelope}s for each input {@link MessageEnvelope}.
+ * The spec for a linear stream operator that outputs 0 or more messages for each input message.
  *
- * @param <M>  the type of input {@link MessageEnvelope}
- * @param <OM>  the type of output {@link MessageEnvelope}
+ * @param <M>  the type of input message
+ * @param <OM>  the type of output message
  */
-public class StreamOperatorSpec<M extends MessageEnvelope, OM extends MessageEnvelope> implements OperatorSpec<OM> {
+public class StreamOperatorSpec<M, OM> implements OperatorSpec<OM> {
 
-  private final MessageStreamImpl<OM> outputStream;
+  /**
+   * {@link OpCode} for this {@link StreamOperatorSpec}
+   */
+  private final OperatorSpec.OpCode opCode;
 
-  private final FlatMapFunction<M, OM> transformFn;
+  /**
+   * The unique ID for this operator.
+   */
+  private final int opId;
 
   /**
-   * Default constructor for a {@link StreamOperatorSpec}.
-   *
-   * @param transformFn  the transformation function that transforms each input {@link MessageEnvelope} into a collection
-   *                     of output {@link MessageEnvelope}s
+   * The output {@link MessageStreamImpl} from this {@link StreamOperatorSpec}
    */
-  StreamOperatorSpec(FlatMapFunction<M, OM> transformFn) {
-    this(transformFn, new MessageStreamImpl<>());
-  }
+  private final MessageStreamImpl<OM> outputStream;
+
+  /**
+   * Transformation function applied in this {@link StreamOperatorSpec}
+   */
+  private final FlatMapFunction<M, OM> transformFn;
 
   /**
    * Constructor for a {@link StreamOperatorSpec} that accepts an output {@link MessageStreamImpl}.
    *
    * @param transformFn  the transformation function
    * @param outputStream  the output {@link MessageStreamImpl}
+   * @param opCode  the {@link OpCode} for this {@link StreamOperatorSpec}
+   * @param opId  the unique id for this {@link StreamOperatorSpec} in a {@link org.apache.samza.operators.StreamGraph}
    */
-  StreamOperatorSpec(FlatMapFunction<M, OM> transformFn, MessageStreamImpl<OM> outputStream) {
+  StreamOperatorSpec(FlatMapFunction<M, OM> transformFn, MessageStreamImpl outputStream, OperatorSpec.OpCode opCode, int opId) {
     this.outputStream = outputStream;
     this.transformFn = transformFn;
+    this.opCode = opCode;
+    this.opId = opId;
   }
 
   @Override
-  public MessageStreamImpl<OM> getOutputStream() {
+  public MessageStreamImpl<OM> getNextStream() {
     return this.outputStream;
   }
 
   public FlatMapFunction<M, OM> getTransformFn() {
     return this.transformFn;
   }
+
+  public OperatorSpec.OpCode getOpCode() {
+    return this.opCode;
+  }
+
+  public int getOpId() {
+    return this.opId;
+  }
+
+  @Override
+  public void init(Config config, TaskContext context) {
+    this.transformFn.init(config, context);
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java
index cdc02a8..46417ed 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java
@@ -19,29 +19,42 @@
 
 package org.apache.samza.operators.spec;
 
-import org.apache.samza.operators.MessageStream;
 import org.apache.samza.operators.MessageStreamImpl;
-import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.operators.windows.WindowPane;
 import org.apache.samza.operators.windows.internal.WindowInternal;
 
-public class WindowOperatorSpec<M extends MessageEnvelope, K, WK, WV, WM extends WindowPane<WK, WV>> implements OperatorSpec<WM> {
 
-  private final WindowInternal window;
+/**
+ * Default window operator spec object
+ *
+ * @param <M>  the type of input message to the window
+ * @param <WK>  the type of key of the window
+ * @param <WV>  the type of aggregated value in the window output {@link WindowPane}
+ */
+public class WindowOperatorSpec<M, WK, WV> implements OperatorSpec<WindowPane<WK, WV>> {
+
+  private final WindowInternal<M, WK, WV> window;
 
-  private final MessageStreamImpl<WM> outputStream;
+  private final MessageStreamImpl<WindowPane<WK, WV>> outputStream;
 
-  private final String operatorId;
+  private final int opId;
 
 
-  public WindowOperatorSpec(WindowInternal window, String operatorId) {
+  /**
+   * Constructor for {@link WindowOperatorSpec}.
+   *
+   * @param window  the window function
+   * @param outputStream  the output {@link MessageStreamImpl} from this {@link WindowOperatorSpec}
+   * @param opId  auto-generated unique ID of this operator
+   */
+  WindowOperatorSpec(WindowInternal<M, WK, WV> window, MessageStreamImpl<WindowPane<WK, WV>> outputStream, int opId) {
+    this.outputStream = outputStream;
     this.window = window;
-    this.outputStream = new MessageStreamImpl<>();
-    this.operatorId = operatorId;
+    this.opId = opId;
   }
 
   @Override
-  public MessageStream<WM> getOutputStream() {
+  public MessageStreamImpl<WindowPane<WK, WV>> getNextStream() {
     return this.outputStream;
   }
 
@@ -49,7 +62,11 @@ public class WindowOperatorSpec<M extends MessageEnvelope, K, WK, WV, WM extends
     return window;
   }
 
-  public String getOperatorId() {
-    return operatorId;
+  public OpCode getOpCode() {
+    return OpCode.WINDOW;
+  }
+
+  public int getOpId() {
+    return this.opId;
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowState.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowState.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowState.java
index e9af043..53bca2e 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowState.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowState.java
@@ -30,20 +30,16 @@ import org.apache.samza.annotation.InterfaceStability;
 @InterfaceStability.Unstable
 public interface WindowState<WV> {
   /**
-   * Method to get the system time when the first {@link org.apache.samza.operators.data.MessageEnvelope}
-   * in the window is received
+   * Method to get the system time when the first message in the window is received
    *
-   * @return  nano-second of system time for the first {@link org.apache.samza.operators.data.MessageEnvelope}
-   *          received in the window
+   * @return  nano-second of system time for the first message received in the window
    */
   long getFirstMessageTimeNs();
 
   /**
-   * Method to get the system time when the last {@link org.apache.samza.operators.data.MessageEnvelope}
-   * in the window is received
+   * Method to get the system time when the last message in the window is received
    *
-   * @return  nano-second of system time for the last {@link org.apache.samza.operators.data.MessageEnvelope}
-   *          received in the window
+   * @return  nano-second of system time for the last message received in the window
    */
   long getLastMessageTimeNs();
 
@@ -62,9 +58,9 @@ public interface WindowState<WV> {
   long getLatestEventTimeNs();
 
   /**
-   * Method to get the total number of {@link org.apache.samza.operators.data.MessageEnvelope}s received in the window
+   * Method to get the total number of messages received in the window
    *
-   * @return  number of {@link org.apache.samza.operators.data.MessageEnvelope}s in the window
+   * @return  number of messages in the window
    */
   long getNumberMessages();
 

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java b/samza-operator/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java
new file mode 100644
index 0000000..fafa2cb
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.system;
+
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.apache.samza.config.Config;
+
+/**
+ * This class implements the {@link ExecutionEnvironment} that runs the applications in YARN environment
+ */
+public class RemoteExecutionEnvironment implements ExecutionEnvironment {
+
+  @Override public void run(StreamGraphBuilder app, Config config) {
+    // TODO: add description of ProcessContext that is going to create a sub-DAG of the {@code graph}
+    // TODO: actually instantiate the tasks and run the job, i.e.
+    // 1. create all input/output/intermediate topics
+    // 2. create the single job configuration
+    // 3. execute JobRunner to submit the single job for the whole graph
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java b/samza-operator/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java
new file mode 100644
index 0000000..f0f6ef2
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.system;
+
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.StreamGraphImpl;
+
+
+/**
+ * This class implements the {@link ExecutionEnvironment} that runs the applications in standalone environment
+ */
+public class StandaloneExecutionEnvironment implements ExecutionEnvironment {
+
+  // TODO: may want to move this to a common base class for all {@link ExecutionEnvironment}
+  StreamGraph createGraph(StreamGraphBuilder app, Config config) {
+    StreamGraphImpl graph = new StreamGraphImpl();
+    app.init(graph, config);
+    return graph;
+  }
+
+  @Override public void run(StreamGraphBuilder app, Config config) {
+    // 1. get logic graph for optimization
+    // StreamGraph logicGraph = this.createGraph(app, config);
+    // 2. potential optimization....
+    // 3. create new instance of StreamGraphBuilder that would generate the optimized graph
+    // 4. create all input/output/intermediate topics
+    // 5. create the configuration for StreamProcessor
+    // 6. start the StreamProcessor w/ optimized instance of StreamGraphBuilder
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java b/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java
new file mode 100644
index 0000000..b007e3c
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.task;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.ContextManager;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.apache.samza.operators.StreamGraphImpl;
+import org.apache.samza.operators.data.InputMessageEnvelope;
+import org.apache.samza.operators.impl.OperatorGraph;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.SystemStream;
+
+import java.util.HashMap;
+import java.util.Map;
+
+
+/**
+ * Execution of the logic sub-DAG
+ *
+ *
+ * An {@link StreamTask} implementation that receives {@link InputMessageEnvelope}s and propagates them
+ * through the user's stream transformations defined in {@link StreamGraphImpl} using the
+ * {@link org.apache.samza.operators.MessageStream} APIs.
+ * <p>
+ * This class brings all the operator API implementation components together and feeds the
+ * {@link InputMessageEnvelope}s into the transformation chains.
+ * <p>
+ * It accepts an instance of the user implemented factory {@link StreamGraphBuilder} as input parameter of the constructor.
+ * When its own {@link #init(Config, TaskContext)} method is called during startup, it instantiate a user-defined {@link StreamGraphImpl}
+ * from the {@link StreamGraphBuilder}, calls {@link StreamGraphImpl#getContextManager()} to initialize the task-wide context
+ * for the graph, and creates a {@link MessageStreamImpl} corresponding to each of its input
+ * {@link org.apache.samza.system.SystemStreamPartition}s. Each input {@link MessageStreamImpl}
+ * will be corresponding to either an input stream or intermediate stream in {@link StreamGraphImpl}.
+ * <p>
+ * Then, this task calls {@link org.apache.samza.operators.impl.OperatorGraph#init(Map, Config, TaskContext)} for each of the input
+ * {@link MessageStreamImpl}. This instantiates the {@link org.apache.samza.operators.impl.OperatorImpl} DAG
+ * corresponding to the aforementioned {@link org.apache.samza.operators.spec.OperatorSpec} DAG and returns the
+ * root node of the DAG, which this class saves.
+ * <p>
+ * Now that it has the root for the DAG corresponding to each {@link org.apache.samza.system.SystemStreamPartition}, it
+ * can pass the message envelopes received in {@link StreamTask#process(IncomingMessageEnvelope, MessageCollector, TaskCoordinator)}
+ * along to the appropriate root nodes. From then on, each {@link org.apache.samza.operators.impl.OperatorImpl} propagates
+ * its transformed output to the next set of {@link org.apache.samza.operators.impl.OperatorImpl}s.
+ */
+public final class StreamOperatorTask implements StreamTask, InitableTask, WindowableTask, ClosableTask {
+
+  /**
+   * A mapping from each {@link SystemStream} to the root node of its operator chain DAG.
+   */
+  private final OperatorGraph operatorGraph = new OperatorGraph();
+
+  private final StreamGraphBuilder graphBuilder;
+
+  private ContextManager contextManager;
+
+  public StreamOperatorTask(StreamGraphBuilder graphBuilder) {
+    this.graphBuilder = graphBuilder;
+  }
+
+  @Override
+  public final void init(Config config, TaskContext context) throws Exception {
+    // create the MessageStreamsImpl object and initialize app-specific logic DAG within the task
+    StreamGraphImpl streams = new StreamGraphImpl();
+    this.graphBuilder.init(streams, config);
+    // get the context manager of the {@link StreamGraph} and initialize the task-specific context
+    this.contextManager = streams.getContextManager();
+
+    Map<SystemStream, MessageStreamImpl> inputBySystemStream = new HashMap<>();
+    context.getSystemStreamPartitions().forEach(ssp -> {
+        if (!inputBySystemStream.containsKey(ssp.getSystemStream())) {
+          // create mapping from the physical input {@link SystemStream} to the logic {@link MessageStream}
+          inputBySystemStream.putIfAbsent(ssp.getSystemStream(), streams.getInputStream(ssp.getSystemStream()));
+        }
+      });
+    operatorGraph.init(inputBySystemStream, config, this.contextManager.initTaskContext(config, context));
+  }
+
+  @Override
+  public final void process(IncomingMessageEnvelope ime, MessageCollector collector, TaskCoordinator coordinator) {
+    this.operatorGraph.get(ime.getSystemStreamPartition().getSystemStream())
+        .onNext(new InputMessageEnvelope(ime), collector, coordinator);
+  }
+
+  @Override
+  public final void window(MessageCollector collector, TaskCoordinator coordinator) throws Exception {
+    // TODO: invoke timer based triggers
+  }
+
+  @Override
+  public void close() throws Exception {
+    this.contextManager.finalizeTaskContext();
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/test/java/org/apache/samza/example/KeyValueStoreExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/KeyValueStoreExample.java b/samza-operator/src/test/java/org/apache/samza/example/KeyValueStoreExample.java
new file mode 100644
index 0000000..85ebc6c
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/KeyValueStoreExample.java
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.samza.operators.*;
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.serializers.JsonSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.storage.kv.KeyValueStore;
+import org.apache.samza.system.ExecutionEnvironment;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.util.CommandLine;
+
+import java.util.Properties;
+
+
+/**
+ * Example code using {@link KeyValueStore} to implement event-time window
+ */
+public class KeyValueStoreExample implements StreamGraphBuilder {
+
+  /**
+   * used by remote execution environment to launch the job in remote program. The remote program should follow the similar
+   * invoking context as in standalone:
+   *
+   *   public static void main(String args[]) throws Exception {
+   *     CommandLine cmdLine = new CommandLine();
+   *     Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+   *     ExecutionEnvironment remoteEnv = ExecutionEnvironment.getRemoteEnvironment(config);
+   *     UserMainExample runnableApp = new UserMainExample();
+   *     runnableApp.run(remoteEnv, config);
+   *   }
+   *
+   */
+  @Override public void init(StreamGraph graph, Config config) {
+
+    MessageStream<PageViewEvent> pageViewEvents = graph.createInStream(input1, new StringSerde("UTF-8"), new JsonSerde<>());
+    OutputStream<StatsOutput> pageViewPerMemberCounters = graph.createOutStream(output, new StringSerde("UTF-8"), new JsonSerde<StatsOutput>());
+
+    pageViewEvents.
+        partitionBy(m -> m.getMessage().memberId).
+        flatMap(new MyStatsCounter()).
+        sendTo(pageViewPerMemberCounters);
+
+  }
+
+  // standalone local program model
+  public static void main(String[] args) throws Exception {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+    ExecutionEnvironment standaloneEnv = ExecutionEnvironment.getLocalEnvironment(config);
+    standaloneEnv.run(new KeyValueStoreExample(), config);
+  }
+
+  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.getMessage().timestamp) / 5) * 5;
+      String wndKey = String.format("%s-%d", message.getMessage().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.getMessage().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");
+    }
+  }
+
+  StreamSpec input1 = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "PageViewEvent");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  StreamSpec output = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "PageViewPerMember5min");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  class PageViewEvent implements MessageEnvelope<String, PageViewEvent> {
+    String pageId;
+    String memberId;
+    long timestamp;
+
+    PageViewEvent(String pageId, String memberId, long timestamp) {
+      this.pageId = pageId;
+      this.memberId = memberId;
+      this.timestamp = timestamp;
+    }
+
+    @Override
+    public String getKey() {
+      return this.pageId;
+    }
+
+    @Override
+    public PageViewEvent getMessage() {
+      return this;
+    }
+  }
+
+  class StatsOutput implements MessageEnvelope<String, 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;
+    }
+
+    @Override
+    public String getKey() {
+      return this.memberId;
+    }
+
+    @Override
+    public StatsOutput getMessage() {
+      return this;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/test/java/org/apache/samza/example/NoContextStreamExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/NoContextStreamExample.java b/samza-operator/src/test/java/org/apache/samza/example/NoContextStreamExample.java
new file mode 100644
index 0000000..c6d2e6e
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/NoContextStreamExample.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.example;
+
+import org.apache.samza.operators.*;
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.data.InputMessageEnvelope;
+import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.Offset;
+import org.apache.samza.operators.functions.JoinFunction;
+import org.apache.samza.serializers.JsonSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.system.ExecutionEnvironment;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.util.CommandLine;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+
+/**
+ * Example {@link StreamGraphBuilder} code to test the API methods
+ */
+public class NoContextStreamExample implements StreamGraphBuilder {
+
+  StreamSpec input1 = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "input1");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  StreamSpec input2 = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "input2");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  StreamSpec output = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "output");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  class MessageType {
+    String joinKey;
+    List<String> joinFields = new ArrayList<>();
+  }
+
+  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
+    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
+      super(key, data, offset, partition);
+    }
+  }
+
+  private JsonMessageEnvelope getInputMessage(InputMessageEnvelope ism) {
+    return new JsonMessageEnvelope(
+        ((MessageType) ism.getMessage()).joinKey,
+        (MessageType) ism.getMessage(),
+        ism.getOffset(),
+        ism.getSystemStreamPartition());
+  }
+
+  class MyJoinFunction implements JoinFunction<String, JsonMessageEnvelope, JsonMessageEnvelope, JsonIncomingSystemMessageEnvelope<MessageType>> {
+
+    @Override
+    public JsonIncomingSystemMessageEnvelope<MessageType> apply(JsonMessageEnvelope m1,
+        JsonMessageEnvelope m2) {
+      MessageType newJoinMsg = new MessageType();
+      newJoinMsg.joinKey = m1.getKey();
+      newJoinMsg.joinFields.addAll(m1.getMessage().joinFields);
+      newJoinMsg.joinFields.addAll(m2.getMessage().joinFields);
+      return new JsonMessageEnvelope(m1.getMessage().joinKey, newJoinMsg, null, null);
+    }
+
+    @Override
+    public String getFirstKey(JsonMessageEnvelope message) {
+      return message.getKey();
+    }
+
+    @Override
+    public String getSecondKey(JsonMessageEnvelope message) {
+      return message.getKey();
+    }
+  }
+
+  /**
+   * used by remote execution environment to launch the job in remote program. The remote program should follow the similar
+   * invoking context as in standalone:
+   *
+   *   public static void main(String args[]) throws Exception {
+   *     CommandLine cmdLine = new CommandLine();
+   *     Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+   *     ExecutionEnvironment remoteEnv = ExecutionEnvironment.fromConfig(config);
+   *     remoteEnv.run(new NoContextStreamExample(), config);
+   *   }
+   *
+   */
+  @Override public void init(StreamGraph graph, Config config) {
+    MessageStream<InputMessageEnvelope> inputSource1 = graph.<Object, Object, InputMessageEnvelope>createInStream(
+        input1, null, null);
+    MessageStream<InputMessageEnvelope> inputSource2 = graph.<Object, Object, InputMessageEnvelope>createInStream(
+        input2, null, null);
+    OutputStream<JsonIncomingSystemMessageEnvelope<MessageType>> outStream = graph.createOutStream(output,
+        new StringSerde("UTF-8"), new JsonSerde<>());
+
+    inputSource1.map(this::getInputMessage).
+        join(inputSource2.map(this::getInputMessage), new MyJoinFunction()).
+        sendTo(outStream);
+
+  }
+
+  // standalone local program model
+  public static void main(String[] args) throws Exception {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+    ExecutionEnvironment standaloneEnv = ExecutionEnvironment.getLocalEnvironment(config);
+    standaloneEnv.run(new NoContextStreamExample(), config);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java b/samza-operator/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java
new file mode 100644
index 0000000..0477066
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.operators.MessageStream;
+import org.apache.samza.operators.OutputStream;
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamSpec;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.functions.JoinFunction;
+import org.apache.samza.serializers.JsonSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.system.ExecutionEnvironment;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.CommandLine;
+
+import java.util.Properties;
+
+
+/**
+ * Simple 2-way stream-to-stream join example
+ */
+public class OrderShipmentJoinExample implements StreamGraphBuilder {
+
+  /**
+   * used by remote execution environment to launch the job in remote program. The remote program should follow the similar
+   * invoking context as in standalone:
+   *
+   *   public static void main(String args[]) throws Exception {
+   *     CommandLine cmdLine = new CommandLine();
+   *     Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+   *     ExecutionEnvironment remoteEnv = ExecutionEnvironment.getRemoteEnvironment(config);
+   *     UserMainExample runnableApp = new UserMainExample();
+   *     runnableApp.run(remoteEnv, config);
+   *   }
+   *
+   */
+  @Override public void init(StreamGraph graph, Config config) {
+
+    MessageStream<OrderRecord> orders = graph.createInStream(input1, new StringSerde("UTF-8"), new JsonSerde<>());
+    MessageStream<ShipmentRecord> shipments = graph.createInStream(input2, new StringSerde("UTF-8"), new JsonSerde<>());
+    OutputStream<FulFilledOrderRecord> fulfilledOrders = graph.createOutStream(output, new StringSerde("UTF-8"), new JsonSerde<>());
+
+    orders.join(shipments, new MyJoinFunction()).sendTo(fulfilledOrders);
+
+  }
+
+  // standalone local program model
+  public static void main(String[] args) throws Exception {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+    ExecutionEnvironment standaloneEnv = ExecutionEnvironment.getLocalEnvironment(config);
+    standaloneEnv.run(new OrderShipmentJoinExample(), config);
+  }
+
+  StreamSpec input1 = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "Orders");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  StreamSpec input2 = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "Shipment");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  StreamSpec output = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "FulfilledOrders");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  class OrderRecord implements MessageEnvelope<String, OrderRecord> {
+    String orderId;
+    long orderTimeMs;
+
+    OrderRecord(String orderId, long timeMs) {
+      this.orderId = orderId;
+      this.orderTimeMs = timeMs;
+    }
+
+    @Override
+    public String getKey() {
+      return this.orderId;
+    }
+
+    @Override
+    public OrderRecord getMessage() {
+      return this;
+    }
+  }
+
+  class ShipmentRecord implements MessageEnvelope<String, ShipmentRecord> {
+    String orderId;
+    long shipTimeMs;
+
+    ShipmentRecord(String orderId, long timeMs) {
+      this.orderId = orderId;
+      this.shipTimeMs = timeMs;
+    }
+
+    @Override
+    public String getKey() {
+      return this.orderId;
+    }
+
+    @Override
+    public ShipmentRecord getMessage() {
+      return this;
+    }
+  }
+
+  class FulFilledOrderRecord implements MessageEnvelope<String, FulFilledOrderRecord> {
+    String orderId;
+    long orderTimeMs;
+    long shipTimeMs;
+
+    FulFilledOrderRecord(String orderId, long orderTimeMs, long shipTimeMs) {
+      this.orderId = orderId;
+      this.orderTimeMs = orderTimeMs;
+      this.shipTimeMs = shipTimeMs;
+    }
+
+
+    @Override
+    public String getKey() {
+      return this.orderId;
+    }
+
+    @Override
+    public FulFilledOrderRecord getMessage() {
+      return this;
+    }
+  }
+
+  FulFilledOrderRecord myJoinResult(OrderRecord m1, ShipmentRecord m2) {
+    return new FulFilledOrderRecord(m1.getKey(), m1.orderTimeMs, m2.shipTimeMs);
+  }
+
+  class MyJoinFunction implements JoinFunction<String, OrderRecord, ShipmentRecord, FulFilledOrderRecord> {
+
+    @Override
+    public FulFilledOrderRecord apply(OrderRecord message, ShipmentRecord otherMessage) {
+      return OrderShipmentJoinExample.this.myJoinResult(message, otherMessage);
+    }
+
+    @Override
+    public String getFirstKey(OrderRecord message) {
+      return message.getKey();
+    }
+
+    @Override
+    public String getSecondKey(ShipmentRecord message) {
+      return message.getKey();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/test/java/org/apache/samza/example/PageViewCounterExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/PageViewCounterExample.java b/samza-operator/src/test/java/org/apache/samza/example/PageViewCounterExample.java
new file mode 100644
index 0000000..f7d8bda
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/PageViewCounterExample.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.example;
+
+import org.apache.samza.operators.*;
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.data.MessageEnvelope;
+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.serializers.JsonSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.system.ExecutionEnvironment;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.CommandLine;
+
+import java.time.Duration;
+import java.util.Properties;
+
+
+/**
+ * Example code to implement window-based counter
+ */
+public class PageViewCounterExample implements StreamGraphBuilder {
+
+  @Override public void init(StreamGraph graph, Config config) {
+
+    MessageStream<PageViewEvent> pageViewEvents = graph.createInStream(input1, new StringSerde("UTF-8"), new JsonSerde<>());
+    OutputStream<MyStreamOutput> pageViewPerMemberCounters = graph.createOutStream(output, new StringSerde("UTF-8"), new JsonSerde<>());
+
+    pageViewEvents.
+        window(Windows.<PageViewEvent, String, Integer>keyedTumblingWindow(m -> m.getMessage().memberId, Duration.ofSeconds(10), (m, c) -> c + 1).
+            setEarlyTrigger(Triggers.repeat(Triggers.count(5))).
+            setAccumulationMode(AccumulationMode.DISCARDING)).
+        map(MyStreamOutput::new).
+        sendTo(pageViewPerMemberCounters);
+
+  }
+
+  public static void main(String[] args) {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+    ExecutionEnvironment standaloneEnv = ExecutionEnvironment.getLocalEnvironment(config);
+    standaloneEnv.run(new PageViewCounterExample(), config);
+  }
+
+  StreamSpec input1 = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "PageViewEvent");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  StreamSpec output = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "PageViewPerMember5min");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  class PageViewEvent implements MessageEnvelope<String, PageViewEvent> {
+    String pageId;
+    String memberId;
+    long timestamp;
+
+    PageViewEvent(String pageId, String memberId, long timestamp) {
+      this.pageId = pageId;
+      this.memberId = memberId;
+      this.timestamp = timestamp;
+    }
+
+    @Override
+    public String getKey() {
+      return this.pageId;
+    }
+
+    @Override
+    public PageViewEvent getMessage() {
+      return this;
+    }
+  }
+
+  class MyStreamOutput implements MessageEnvelope<String, 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();
+    }
+
+    @Override
+    public String getKey() {
+      return this.memberId;
+    }
+
+    @Override
+    public MyStreamOutput getMessage() {
+      return this;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/test/java/org/apache/samza/example/RepartitionExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/RepartitionExample.java b/samza-operator/src/test/java/org/apache/samza/example/RepartitionExample.java
new file mode 100644
index 0000000..6994ac4
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/RepartitionExample.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.operators.*;
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.windows.WindowPane;
+import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.serializers.JsonSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.system.ExecutionEnvironment;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.CommandLine;
+
+import java.time.Duration;
+import java.util.*;
+
+
+/**
+ * Example {@link StreamGraphBuilder} code to test the API methods with re-partition operator
+ */
+public class RepartitionExample implements StreamGraphBuilder {
+
+  /**
+   * used by remote execution environment to launch the job in remote program. The remote program should follow the similar
+   * invoking context as in standalone:
+   *
+   *   public static void main(String args[]) throws Exception {
+   *     CommandLine cmdLine = new CommandLine();
+   *     Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+   *     ExecutionEnvironment remoteEnv = ExecutionEnvironment.getRemoteEnvironment(config);
+   *     remoteEnv.run(new UserMainExample(), config);
+   *   }
+   *
+   */
+  @Override public void init(StreamGraph graph, Config config) {
+
+    MessageStream<PageViewEvent> pageViewEvents = graph.createInStream(input1, new StringSerde("UTF-8"), new JsonSerde<>());
+    OutputStream<MyStreamOutput> pageViewPerMemberCounters = graph.createOutStream(output, new StringSerde("UTF-8"), new JsonSerde<>());
+
+    pageViewEvents.
+        partitionBy(m -> m.getMessage().memberId).
+        window(Windows.<PageViewEvent, String, Integer>keyedTumblingWindow(
+            msg -> msg.getMessage().memberId, Duration.ofMinutes(5), (m, c) -> c + 1)).
+        map(MyStreamOutput::new).
+        sendTo(pageViewPerMemberCounters);
+
+  }
+
+  // standalone local program model
+  public static void main(String[] args) throws Exception {
+    CommandLine cmdLine = new CommandLine();
+    Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+    ExecutionEnvironment standaloneEnv = ExecutionEnvironment.getLocalEnvironment(config);
+    standaloneEnv.run(new RepartitionExample(), config);
+  }
+
+  StreamSpec input1 = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "PageViewEvent");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  StreamSpec output = new StreamSpec() {
+    @Override public SystemStream getSystemStream() {
+      return new SystemStream("kafka", "PageViewPerMember5min");
+    }
+
+    @Override public Properties getProperties() {
+      return null;
+    }
+  };
+
+  class PageViewEvent implements MessageEnvelope<String, PageViewEvent> {
+    String pageId;
+    String memberId;
+    long timestamp;
+
+    PageViewEvent(String pageId, String memberId, long timestamp) {
+      this.pageId = pageId;
+      this.memberId = memberId;
+      this.timestamp = timestamp;
+    }
+
+    @Override
+    public String getKey() {
+      return this.pageId;
+    }
+
+    @Override
+    public PageViewEvent getMessage() {
+      return this;
+    }
+  }
+
+  class MyStreamOutput implements MessageEnvelope<String, 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();
+    }
+
+    @Override
+    public String getKey() {
+      return this.memberId;
+    }
+
+    @Override
+    public MyStreamOutput getMessage() {
+      return this;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/test/java/org/apache/samza/example/TestBasicStreamGraphs.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/TestBasicStreamGraphs.java b/samza-operator/src/test/java/org/apache/samza/example/TestBasicStreamGraphs.java
new file mode 100644
index 0000000..8ecd44f
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/TestBasicStreamGraphs.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.lang.reflect.Field;
+import org.apache.samza.Partition;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.impl.OperatorGraph;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.task.StreamOperatorTask;
+import org.apache.samza.task.TaskContext;
+import org.junit.Test;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+
+/**
+ * Unit test for {@link StreamOperatorTask}
+ */
+public class TestBasicStreamGraphs {
+
+  private final Set<SystemStreamPartition> inputPartitions = new HashSet<SystemStreamPartition>() { {
+      for (int i = 0; i < 4; i++) {
+        this.add(new SystemStreamPartition("my-system", String.format("my-topic%d", i), new Partition(i)));
+      }
+    } };
+
+  @Test
+  public void testUserTask() throws Exception {
+    Config mockConfig = mock(Config.class);
+    TaskContext mockContext = mock(TaskContext.class);
+    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
+    TestWindowExample userTask = new TestWindowExample(this.inputPartitions);
+    StreamOperatorTask adaptorTask = new StreamOperatorTask(userTask);
+    Field pipelineMapFld = StreamOperatorTask.class.getDeclaredField("operatorGraph");
+    pipelineMapFld.setAccessible(true);
+    OperatorGraph opGraph = (OperatorGraph) pipelineMapFld.get(adaptorTask);
+
+    adaptorTask.init(mockConfig, mockContext);
+    this.inputPartitions.forEach(partition -> {
+        assertNotNull(opGraph.get(partition.getSystemStream()));
+      });
+  }
+
+  @Test
+  public void testSplitTask() throws Exception {
+    Config mockConfig = mock(Config.class);
+    TaskContext mockContext = mock(TaskContext.class);
+    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
+    TestBroadcastExample splitTask = new TestBroadcastExample(this.inputPartitions);
+    StreamOperatorTask adaptorTask = new StreamOperatorTask(splitTask);
+    Field pipelineMapFld = StreamOperatorTask.class.getDeclaredField("operatorGraph");
+    pipelineMapFld.setAccessible(true);
+    OperatorGraph opGraph = (OperatorGraph) pipelineMapFld.get(adaptorTask);
+
+    adaptorTask.init(mockConfig, mockContext);
+    this.inputPartitions.forEach(partition -> {
+        assertNotNull(opGraph.get(partition.getSystemStream()));
+      });
+  }
+
+  @Test
+  public void testJoinTask() throws Exception {
+    Config mockConfig = mock(Config.class);
+    TaskContext mockContext = mock(TaskContext.class);
+    when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
+    TestJoinExample joinTask = new TestJoinExample(this.inputPartitions);
+    StreamOperatorTask adaptorTask = new StreamOperatorTask(joinTask);
+    Field pipelineMapFld = StreamOperatorTask.class.getDeclaredField("operatorGraph");
+    pipelineMapFld.setAccessible(true);
+    OperatorGraph opGraph = (OperatorGraph) pipelineMapFld.get(adaptorTask);
+
+    adaptorTask.init(mockConfig, mockContext);
+    this.inputPartitions.forEach(partition -> {
+        assertNotNull(opGraph.get(partition.getSystemStream()));
+      });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/test/java/org/apache/samza/example/TestBroadcastExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/TestBroadcastExample.java b/samza-operator/src/test/java/org/apache/samza/example/TestBroadcastExample.java
new file mode 100644
index 0000000..d22324b
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/TestBroadcastExample.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.example;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamSpec;
+import org.apache.samza.operators.data.InputMessageEnvelope;
+import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.Offset;
+import org.apache.samza.operators.triggers.Triggers;
+import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.time.Duration;
+import java.util.function.BiFunction;
+import java.util.Properties;
+import java.util.Set;
+
+
+/**
+ * Example implementation of split stream tasks
+ *
+ */
+public class TestBroadcastExample extends TestExampleBase {
+
+  TestBroadcastExample(Set<SystemStreamPartition> inputs) {
+    super(inputs);
+  }
+
+  class MessageType {
+    String field1;
+    String field2;
+    String field3;
+    String field4;
+    String parKey;
+    private long timestamp;
+
+    public long getTimestamp() {
+      return this.timestamp;
+    }
+  }
+
+  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
+    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
+      super(key, data, offset, partition);
+    }
+  }
+
+  @Override
+  public void init(StreamGraph graph, Config config) {
+    BiFunction<JsonMessageEnvelope, Integer, Integer> sumAggregator = (m, c) -> c + 1;
+    inputs.keySet().forEach(entry -> {
+        MessageStream<JsonMessageEnvelope> inputStream = graph.<Object, Object, InputMessageEnvelope>createInStream(new StreamSpec() {
+          @Override public SystemStream getSystemStream() {
+            return entry;
+          }
+
+          @Override public Properties getProperties() {
+            return null;
+          }
+        }, null, null).map(this::getInputMessage);
+
+        inputStream.filter(this::myFilter1).window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
+            .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
+
+        inputStream.filter(this::myFilter2).window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
+            .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
+
+        inputStream.filter(this::myFilter3).window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
+            .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
+
+      });
+  }
+
+  JsonMessageEnvelope getInputMessage(InputMessageEnvelope m1) {
+    return (JsonMessageEnvelope) m1.getMessage();
+  }
+
+  boolean myFilter1(JsonMessageEnvelope m1) {
+    // Do user defined processing here
+    return m1.getMessage().parKey.equals("key1");
+  }
+
+  boolean myFilter2(JsonMessageEnvelope m1) {
+    // Do user defined processing here
+    return m1.getMessage().parKey.equals("key2");
+  }
+
+  boolean myFilter3(JsonMessageEnvelope m1) {
+    return m1.getMessage().parKey.equals("key3");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c249443b/samza-operator/src/test/java/org/apache/samza/example/TestExampleBase.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/TestExampleBase.java b/samza-operator/src/test/java/org/apache/samza/example/TestExampleBase.java
new file mode 100644
index 0000000..c4df9d4
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/TestExampleBase.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.example;
+
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Base class for test examples
+ *
+ */
+public abstract class TestExampleBase implements StreamGraphBuilder {
+
+  protected final Map<SystemStream, Set<SystemStreamPartition>> inputs;
+
+  TestExampleBase(Set<SystemStreamPartition> inputs) {
+    this.inputs = new HashMap<>();
+    for (SystemStreamPartition input : inputs) {
+      this.inputs.putIfAbsent(input.getSystemStream(), new HashSet<>());
+      this.inputs.get(input.getSystemStream()).add(input);
+    }
+  }
+
+}


[02/14] samza git commit: SAMZA-1073: Addressing review feedbacks. Change StreamGraphFactory to StreamGraphBuilder.

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/test/java/org/apache/samza/example/TestWindowExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/TestWindowExample.java b/samza-operator/src/test/java/org/apache/samza/example/TestWindowExample.java
new file mode 100644
index 0000000..e08ca20
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/example/TestWindowExample.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.example;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamSpec;
+import org.apache.samza.operators.data.InputMessageEnvelope;
+import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.data.Offset;
+import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.time.Duration;
+import java.util.function.BiFunction;
+import java.util.Properties;
+import java.util.Set;
+
+
+/**
+ * Example implementation of a simple user-defined tasks w/ window operators
+ *
+ */
+public class TestWindowExample extends TestExampleBase {
+  class MessageType {
+    String field1;
+    String field2;
+  }
+
+  TestWindowExample(Set<SystemStreamPartition> inputs) {
+    super(inputs);
+  }
+
+  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
+
+    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
+      super(key, data, offset, partition);
+    }
+  }
+
+  @Override
+  public void init(StreamGraph graph, Config config) {
+    BiFunction<JsonMessageEnvelope, Integer, Integer> maxAggregator = (m, c) -> c + 1;
+    inputs.keySet().forEach(source -> graph.<Object, Object, InputMessageEnvelope>createInStream(new StreamSpec() {
+      @Override public SystemStream getSystemStream() {
+        return source;
+      }
+
+      @Override public Properties getProperties() {
+        return null;
+      }
+    }, null, null).
+        map(m1 -> new JsonMessageEnvelope(this.myMessageKeyFunction(m1), (MessageType) m1.getMessage(), m1.getOffset(),
+            m1.getSystemStreamPartition())).window(Windows.tumblingWindow(Duration.ofMillis(200), maxAggregator)));
+
+  }
+
+  String myMessageKeyFunction(MessageEnvelope<Object, Object> m) {
+    return m.getKey().toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/test/java/org/apache/samza/example/WindowGraph.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/WindowGraph.java b/samza-operator/src/test/java/org/apache/samza/example/WindowGraph.java
deleted file mode 100644
index a365411..0000000
--- a/samza-operator/src/test/java/org/apache/samza/example/WindowGraph.java
+++ /dev/null
@@ -1,87 +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.config.Config;
-import org.apache.samza.operators.StreamGraph;
-import org.apache.samza.operators.StreamGraphFactory;
-import org.apache.samza.operators.StreamGraphImpl;
-import org.apache.samza.operators.StreamSpec;
-import org.apache.samza.operators.data.InputMessageEnvelope;
-import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
-import org.apache.samza.operators.data.MessageEnvelope;
-import org.apache.samza.operators.data.Offset;
-import org.apache.samza.operators.windows.Windows;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.time.Duration;
-import java.util.function.BiFunction;
-import java.util.Properties;
-import java.util.Set;
-
-
-/**
- * Example implementation of a simple user-defined tasks w/ window operators
- *
- */
-public class WindowGraph implements StreamGraphFactory {
-  class MessageType {
-    String field1;
-    String field2;
-  }
-
-  private final Set<SystemStreamPartition> inputs;
-
-  WindowGraph(Set<SystemStreamPartition> inputs) {
-    this.inputs = inputs;
-  }
-
-  class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
-
-    JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
-      super(key, data, offset, partition);
-    }
-  }
-
-  @Override
-  public StreamGraph create(Config config) {
-    StreamGraphImpl graph = new StreamGraphImpl();
-    BiFunction<JsonMessageEnvelope, Integer, Integer> maxAggregator = (m, c) -> c + 1;
-    inputs.forEach(source -> graph.<Object, Object, InputMessageEnvelope>createInStream(new StreamSpec() {
-      @Override public SystemStream getSystemStream() {
-        return source.getSystemStream();
-      }
-
-      @Override public Properties getProperties() {
-        return null;
-      }
-    }, null, null).
-        map(m1 -> new JsonMessageEnvelope(this.myMessageKeyFunction(m1), (MessageType) m1.getMessage(), m1.getOffset(),
-            m1.getSystemStreamPartition())).window(Windows.tumblingWindow(Duration.ofMillis(200), maxAggregator)));
-
-    return graph;
-  }
-
-  String myMessageKeyFunction(MessageEnvelope<Object, Object> m) {
-    return m.getKey().toString();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
index d5607d8..160a47a 100644
--- a/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
+++ b/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
@@ -21,7 +21,6 @@ package org.apache.samza.operators;
 import org.apache.samza.operators.functions.FilterFunction;
 import org.apache.samza.operators.functions.FlatMapFunction;
 import org.apache.samza.operators.functions.JoinFunction;
-import org.apache.samza.operators.functions.KeyValueJoinFunction;
 import org.apache.samza.operators.functions.MapFunction;
 import org.apache.samza.operators.functions.SinkFunction;
 import org.apache.samza.operators.spec.OperatorSpec;
@@ -140,8 +139,22 @@ public class TestMessageStreamImpl {
     MessageStreamImpl<TestMessageEnvelope> source1 = new MessageStreamImpl<>(mockGraph);
     MessageStreamImpl<TestMessageEnvelope> source2 = new MessageStreamImpl<>(mockGraph);
     JoinFunction<String, TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope> joiner =
-        (KeyValueJoinFunction<String, TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope>)
-            (m1, m2) -> new TestOutputMessageEnvelope(m1.getKey(), m1.getMessage().getValue().length() + m2.getMessage().getValue().length());
+      new JoinFunction<String, TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope>() {
+        @Override
+        public TestOutputMessageEnvelope apply(TestMessageEnvelope m1, TestMessageEnvelope m2) {
+          return new TestOutputMessageEnvelope(m1.getKey(), m1.getMessage().getValue().length() + m2.getMessage().getValue().length());
+        }
+
+        @Override
+        public String getFirstKey(TestMessageEnvelope message) {
+          return message.getKey();
+        }
+
+        @Override
+        public String getSecondKey(TestMessageEnvelope message) {
+          return message.getKey();
+        }
+      };
 
     MessageStream<TestOutputMessageEnvelope> joinOutput = source1.join(source2, joiner);
     Collection<OperatorSpec> subs = source1.getRegisteredOperatorSpecs();

http://git-wip-us.apache.org/repos/asf/samza/blob/4bde68b4/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java
index ec63d41..02637a3 100644
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java
@@ -28,7 +28,7 @@ import org.apache.samza.operators.TestMessageStreamImplUtil;
 import org.apache.samza.operators.TestOutputMessageEnvelope;
 import org.apache.samza.operators.data.MessageEnvelope;
 import org.apache.samza.operators.functions.FlatMapFunction;
-import org.apache.samza.operators.functions.KeyValueJoinFunction;
+import org.apache.samza.operators.functions.JoinFunction;
 import org.apache.samza.operators.functions.PartialJoinFunction;
 import org.apache.samza.operators.functions.SinkFunction;
 import org.apache.samza.operators.spec.OperatorSpec;
@@ -192,9 +192,22 @@ public class TestOperatorImpls {
     Config mockConfig = mock(Config.class);
     input1
         .join(input2,
-            (KeyValueJoinFunction<String, TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope>) (m1, m2) ->
-                new TestOutputMessageEnvelope(m1.getKey(), m1.getMessage().getValue().length() + m2.getMessage().getValue().length())
-            )
+            new JoinFunction<String, TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope>() {
+              @Override
+              public TestOutputMessageEnvelope apply(TestMessageEnvelope m1, TestMessageEnvelope m2) {
+                return new TestOutputMessageEnvelope(m1.getKey(), m1.getMessage().getValue().length() + m2.getMessage().getValue().length());
+              }
+
+              @Override
+              public String getFirstKey(TestMessageEnvelope message) {
+                return message.getKey();
+              }
+
+              @Override
+              public String getSecondKey(TestMessageEnvelope message) {
+                return message.getKey();
+              }
+            })
         .map(m -> m);
     OperatorGraph opGraph = new OperatorGraph();
     // now, we create chained operators from each input sources