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

[1/5] samza git commit: SAMZA-914: Creating the fluent programming APIs w/ operators

Repository: samza
Updated Branches:
  refs/heads/samza-sql f29c61498 -> fbdd76daa


http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/operators/join/StreamStreamJoin.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/join/StreamStreamJoin.java b/samza-sql-core/src/main/java/org/apache/samza/sql/operators/join/StreamStreamJoin.java
deleted file mode 100644
index 2854aeb..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/join/StreamStreamJoin.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.operators.join;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.sql.api.data.EntityName;
-import org.apache.samza.sql.api.data.Relation;
-import org.apache.samza.sql.api.data.Stream;
-import org.apache.samza.sql.api.data.Tuple;
-import org.apache.samza.sql.api.operators.OperatorCallback;
-import org.apache.samza.sql.operators.factory.SimpleOperatorImpl;
-import org.apache.samza.sql.operators.window.BoundedTimeWindow;
-import org.apache.samza.sql.window.storage.OrderedStoreKey;
-import org.apache.samza.storage.kv.Entry;
-import org.apache.samza.storage.kv.KeyValueIterator;
-import org.apache.samza.task.TaskContext;
-import org.apache.samza.task.TaskCoordinator;
-import org.apache.samza.task.sql.SimpleMessageCollector;
-
-
-/**
- * This class implements a simple stream-to-stream join
- */
-public class StreamStreamJoin extends SimpleOperatorImpl {
-  private final StreamStreamJoinSpec spec;
-
-  private Map<EntityName, BoundedTimeWindow> inputWindows = new HashMap<EntityName, BoundedTimeWindow>();
-
-  public StreamStreamJoin(StreamStreamJoinSpec spec) {
-    super(spec);
-    this.spec = spec;
-  }
-
-  //TODO: stub constructor to allow compilation pass. Need to construct real StreamStreamJoinSpec.
-  public StreamStreamJoin(String opId, List<String> inputRelations, String output, List<String> joinKeys) {
-    this(null);
-  }
-
-  //TODO: stub constructor to allow compilation pass. Need to construct real StreamStreamJoinSpec.
-  public StreamStreamJoin(String opId, List<String> inputRelations, String output, List<String> joinKeys,
-      OperatorCallback callback) {
-    super(null, callback);
-    this.spec = null;
-  }
-
-  @Override
-  public void init(Config config, TaskContext context) throws Exception {
-    // TODO Auto-generated method stub
-    // initialize the inputWindows map
-
-  }
-
-  private void join(Tuple tuple, Map<EntityName, Stream> joinSets) {
-    // TODO Auto-generated method stub
-    // Do M-way joins if necessary, it should be ordered based on the orders of the input relations in inputs
-    // NOTE: inner joins may be optimized by re-order the input relations by joining inputs w/ less join sets first. We will consider it later.
-
-  }
-
-  private Map<EntityName, Stream> findJoinSets(Tuple tuple) {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  private KeyValueIterator<OrderedStoreKey, Tuple> getJoinSet(Tuple tuple, EntityName strmName) {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  private List<Entry<String, Object>> getEqualFields(Tuple tuple, EntityName strmName) {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  @Override
-  protected void realProcess(Relation deltaRelation, SimpleMessageCollector collector, TaskCoordinator coordinator)
-      throws Exception {
-    // TODO Auto-generated method stub
-
-  }
-
-  @Override
-  protected void realProcess(Tuple tuple, SimpleMessageCollector collector, TaskCoordinator coordinator)
-      throws Exception {
-    // TODO Auto-generated method stub
-    Map<EntityName, Stream> joinSets = findJoinSets(tuple);
-    join(tuple, joinSets);
-  }
-
-  @Override
-  public void realRefresh(long timeNano, SimpleMessageCollector collector, TaskCoordinator coordinator)
-      throws Exception {
-    // TODO Auto-generated method stub
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/operators/join/StreamStreamJoinSpec.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/join/StreamStreamJoinSpec.java b/samza-sql-core/src/main/java/org/apache/samza/sql/operators/join/StreamStreamJoinSpec.java
deleted file mode 100644
index cc0aca0..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/join/StreamStreamJoinSpec.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.sql.operators.join;
-
-import java.util.List;
-
-import org.apache.samza.sql.api.data.EntityName;
-import org.apache.samza.sql.operators.factory.SimpleOperatorSpec;
-
-
-/**
- * This class defines the specification of a {@link org.apache.samza.sql.operators.join.StreamStreamJoin} operator
- */
-public class StreamStreamJoinSpec extends SimpleOperatorSpec {
-
-  public StreamStreamJoinSpec(String id, List<EntityName> inputs, EntityName output, List<String> joinKeys) {
-    super(id, inputs, output);
-    // TODO Auto-generated constructor stub
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java b/samza-sql-core/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java
deleted file mode 100644
index b93d789..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.operators.partition;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.sql.api.data.Relation;
-import org.apache.samza.sql.api.data.Tuple;
-import org.apache.samza.sql.api.operators.OperatorCallback;
-import org.apache.samza.sql.operators.factory.SimpleOperatorImpl;
-import org.apache.samza.storage.kv.Entry;
-import org.apache.samza.storage.kv.KeyValueIterator;
-import org.apache.samza.system.OutgoingMessageEnvelope;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.task.TaskContext;
-import org.apache.samza.task.TaskCoordinator;
-import org.apache.samza.task.sql.SimpleMessageCollector;
-
-
-/**
- * This is an example build-in operator that performs a simple stream re-partition operation.
- *
- */
-public class PartitionOp extends SimpleOperatorImpl {
-
-  /**
-   * The specification of this {@code PartitionOp}
-   *
-   */
-  private final PartitionSpec spec;
-
-  /**
-   * Ctor that takes the {@link org.apache.samza.sql.operators.partition.PartitionSpec} object as input.
-   *
-   * @param spec The <code>PartitionSpec</code> object
-   */
-  public PartitionOp(PartitionSpec spec) {
-    super(spec);
-    this.spec = spec;
-  }
-
-  /**
-   * A simplified constructor that allow users to randomly create <code>PartitionOp</code>
-   *
-   * @param id The identifier of this operator
-   * @param input The input stream name of this operator
-   * @param system The output system name of this operator
-   * @param output The output stream name of this operator
-   * @param parKey The partition key used for the output stream
-   * @param parNum The number of partitions used for the output stream
-   */
-  public PartitionOp(String id, String input, String system, String output, String parKey, int parNum) {
-    this(new PartitionSpec(id, input, new SystemStream(system, output), parKey, parNum));
-  }
-
-  /**
-   * A simplified constructor that allow users to randomly create <code>PartitionOp</code>
-   *
-   * @param id The identifier of this operator
-   * @param input The input stream name of this operator
-   * @param system The output system name of this operator
-   * @param output The output stream name of this operator
-   * @param parKey The partition key used for the output stream
-   * @param parNum The number of partitions used for the output stream
-   * @param callback The callback functions for operator
-   */
-  public PartitionOp(String id, String input, String system, String output, String parKey, int parNum,
-      OperatorCallback callback) {
-    super(new PartitionSpec(id, input, new SystemStream(system, output), parKey, parNum), callback);
-    this.spec = (PartitionSpec) super.getSpec();
-  }
-
-  @Override
-  public void init(Config config, TaskContext context) throws Exception {
-    // TODO Auto-generated method stub
-    // No need to initialize store since all inputs are immediately send out
-  }
-
-  @Override
-  protected void realRefresh(long timeNano, SimpleMessageCollector collector, TaskCoordinator coordinator)
-      throws Exception {
-    // TODO Auto-generated method stub
-    // NOOP or flush
-  }
-
-  @Override
-  protected void realProcess(Tuple tuple, SimpleMessageCollector collector, TaskCoordinator coordinator)
-      throws Exception {
-    collector.send(new OutgoingMessageEnvelope(PartitionOp.this.spec.getSystemStream(), tuple.getMessage()
-        .getFieldData(PartitionOp.this.spec.getParKey()).value(), tuple.getKey().value(), tuple.getMessage().value()));
-  }
-
-  @Override
-  protected void realProcess(Relation deltaRelation, SimpleMessageCollector collector, TaskCoordinator coordinator)
-      throws Exception {
-    for(KeyValueIterator<?, Tuple> iter = deltaRelation.all(); iter.hasNext(); ) {
-      Entry<?, Tuple> entry = iter.next();
-      collector.send(new OutgoingMessageEnvelope(PartitionOp.this.spec.getSystemStream(), entry.getValue().getMessage()
-          .getFieldData(PartitionOp.this.spec.getParKey()).value(), entry.getValue().getKey().value(), entry.getValue()
-          .getMessage().value()));
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java b/samza-sql-core/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java
deleted file mode 100644
index c47eed9..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.operators.partition;
-
-import org.apache.samza.sql.api.data.EntityName;
-import org.apache.samza.sql.api.operators.OperatorSpec;
-import org.apache.samza.sql.operators.factory.SimpleOperatorSpec;
-import org.apache.samza.system.SystemStream;
-
-
-/**
- * This class defines the specification class of {@link org.apache.samza.sql.operators.partition.PartitionOp}
- *
- */
-public class PartitionSpec extends SimpleOperatorSpec implements OperatorSpec {
-
-  /**
-   * The partition key name
-   */
-  private final String parKey;
-
-  /**
-   * The number of partitions
-   */
-  private final int parNum;
-
-  /**
-   * The <code>SystemStream</code> to send the partition output to
-   */
-  private final SystemStream sysStream;
-
-  /**
-   * Ctor to create the {@code PartitionSpec}
-   *
-   * @param id The ID of the {@link org.apache.samza.sql.operators.partition.PartitionOp}
-   * @param input The input stream name
-   * @param output The output {@link org.apache.samza.system.SystemStream} object
-   * @param parKey The name of the partition key
-   * @param parNum The number of partitions
-   */
-  public PartitionSpec(String id, String input, SystemStream output, String parKey, int parNum) {
-    super(id, EntityName.getStreamName(input), EntityName.getStreamName(output.getSystem() + ":" + output.getStream()));
-    this.parKey = parKey;
-    this.parNum = parNum;
-    this.sysStream = output;
-  }
-
-  /**
-   * Method to get the partition key name
-   *
-   * @return The partition key name
-   */
-  public String getParKey() {
-    return this.parKey;
-  }
-
-  /**
-   * Method to get the number of partitions
-   *
-   * @return The number of partitions
-   */
-  public int getParNum() {
-    return this.parNum;
-  }
-
-  /**
-   * Method to get the output {@link org.apache.samza.system.SystemStream}
-   *
-   * @return The output system stream object
-   */
-  public SystemStream getSystemStream() {
-    return this.sysStream;
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java b/samza-sql-core/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java
deleted file mode 100644
index d81cc93..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.operators.window;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.sql.api.data.EntityName;
-import org.apache.samza.sql.api.data.Relation;
-import org.apache.samza.sql.api.data.Tuple;
-import org.apache.samza.sql.api.operators.OperatorCallback;
-import org.apache.samza.sql.operators.factory.SimpleOperatorImpl;
-import org.apache.samza.storage.kv.KeyValueIterator;
-import org.apache.samza.task.TaskContext;
-import org.apache.samza.task.TaskCoordinator;
-import org.apache.samza.task.sql.SimpleMessageCollector;
-
-
-/**
- * This class defines an example build-in operator for a fixed size window operator that converts a stream to a relation
- *
- */
-public class BoundedTimeWindow extends SimpleOperatorImpl {
-
-  /**
-   * The specification of this window operator
-   */
-  private final WindowSpec spec;
-
-  /**
-   * The relation that the window operator keeps internally
-   */
-  private Relation relation = null;
-
-  /**
-   * The list of window states of all active windows the window operator keeps in track
-   */
-  private List<WindowState> windowStates = null;
-
-  /**
-   * Ctor that takes <code>WindowSpec</code> specification as input argument
-   *
-   * <p>This version of constructor is often used in an implementation of {@link org.apache.samza.sql.api.operators.SqlOperatorFactory}
-   *
-   * @param spec The window specification object
-   */
-  public BoundedTimeWindow(WindowSpec spec) {
-    super(spec);
-    this.spec = spec;
-  }
-
-  /**
-   * A simplified version of ctor that allows users to randomly created a window operator w/o spec object
-   *
-   * @param wndId The identifier of this window operator
-   * @param lengthSec The window size in seconds
-   * @param input The input stream name
-   * @param output The output relation name
-   */
-  public BoundedTimeWindow(String wndId, int lengthSec, String input, String output) {
-    this(new WindowSpec(wndId, EntityName.getStreamName(input), EntityName.getStreamName(output), lengthSec));
-  }
-
-  /**
-   * A simplified version of ctor that allows users to randomly created a window operator w/o spec object
-   *
-   * @param wndId The identifier of this window operator
-   * @param lengthSec The window size in seconds
-   * @param input The input stream name
-   * @param output The output relation name
-   */
-  public BoundedTimeWindow(String wndId, int lengthSec, String input, String output, OperatorCallback callback) {
-    super(new WindowSpec(wndId, EntityName.getStreamName(input), EntityName.getStreamName(output), lengthSec), callback);
-    this.spec = (WindowSpec) super.getSpec();
-  }
-
-  private void processWindowChanges(SimpleMessageCollector collector) throws Exception {
-    if (windowStateChange()) {
-      collector.send(getWindowChanges());
-    }
-  }
-
-  private Relation getWindowChanges() {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  private boolean windowStateChange() {
-    // TODO Auto-generated method stub
-    return getWindowChanges() != null;
-  }
-
-  private void updateWindow(Tuple tuple) {
-    // TODO Auto-generated method stub
-    // The window states are updated here
-    // And the correpsonding deltaChanges is also calculated here.
-  }
-
-  private void updateWindowTimeout() {
-    // TODO Auto-generated method stub
-    // The window states are updated here
-    // And the correpsonding deltaChanges is also calculated here.
-  }
-
-  @Override
-  public void init(Config config, TaskContext context) throws Exception {
-    // TODO Auto-generated method stub
-    if (this.relation == null) {
-      this.relation = (Relation) context.getStore(this.spec.getOutputName().toString());
-      Relation wndStates = (Relation) context.getStore(this.spec.getWndStatesName());
-      this.windowStates = new ArrayList<WindowState>();
-      for (KeyValueIterator<Object, Tuple> iter = wndStates.all(); iter.hasNext();) {
-        this.windowStates.add((WindowState) iter.next().getValue().getMessage());
-      }
-    }
-  }
-
-  @Override
-  protected void realProcess(Tuple tuple, SimpleMessageCollector collector, TaskCoordinator coordinator)
-      throws Exception {
-    // for each tuple, this will evaluate the incoming tuple and update the window states.
-    // If the window states allow generating output, calculate the delta changes in
-    // the window relation and execute the relation operation <code>nextOp</code>
-    updateWindow(tuple);
-    processWindowChanges(collector);
-  }
-
-  @Override
-  protected void realProcess(Relation rel, SimpleMessageCollector collector, TaskCoordinator coordinator)
-      throws Exception {
-    for (KeyValueIterator<Object, Tuple> iter = rel.all(); iter.hasNext();) {
-      updateWindow(iter.next().getValue());
-      processWindowChanges(collector);
-    }
-  }
-
-  @Override
-  protected void realRefresh(long timeNano, SimpleMessageCollector collector, TaskCoordinator coordinator)
-      throws Exception {
-    updateWindowTimeout();
-    processWindowChanges(collector);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java b/samza-sql-core/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java
deleted file mode 100644
index eec32ea..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.operators.window;
-
-import org.apache.samza.sql.api.data.EntityName;
-import org.apache.samza.sql.api.operators.OperatorSpec;
-import org.apache.samza.sql.operators.factory.SimpleOperatorSpec;
-
-
-/**
- * This class implements the specification class for the build-in {@link org.apache.samza.sql.operators.window.BoundedTimeWindow} operator
- */
-public class WindowSpec extends SimpleOperatorSpec implements OperatorSpec {
-
-  /**
-   * The window size in seconds
-   */
-  private final int wndSizeSec;
-
-  /**
-   * Default ctor of the {@code WindowSpec} object
-   *
-   * @param id The identifier of the operator
-   * @param input The input stream entity
-   * @param output The output relation entity
-   * @param lengthSec The window size in seconds
-   */
-  public WindowSpec(String id, EntityName input, EntityName output, int lengthSec) {
-    super(id, input, output);
-    this.wndSizeSec = lengthSec;
-  }
-
-  /**
-   * Method to get the window state relation name
-   *
-   * @return The window state relation name
-   */
-  public String getWndStatesName() {
-    return this.getId() + "-wnd-state";
-  }
-
-  /**
-   * Method to get the window size in seconds
-   *
-   * @return The window size in seconds
-   */
-  public int getWndSizeSec() {
-    return this.wndSizeSec;
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/operators/window/WindowState.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/window/WindowState.java b/samza-sql-core/src/main/java/org/apache/samza/sql/operators/window/WindowState.java
deleted file mode 100644
index 48547f0..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/window/WindowState.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.sql.operators.window;
-
-public class WindowState {
-  public String startOffset = null;
-  public String endOffset = null;
-  public boolean isClosed = false;
-
-  public void open(String offset) {
-    this.isClosed = false;
-    this.startOffset = offset;
-  }
-
-  public void close(String offset) {
-    this.endOffset = offset;
-    this.isClosed = true;
-  }
-
-  public void advanceTo(String offset) {
-    this.endOffset = offset;
-  }
-
-  public boolean isClosed() {
-    return this.isClosed;
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/window/storage/OrderedStoreKey.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/window/storage/OrderedStoreKey.java b/samza-sql-core/src/main/java/org/apache/samza/sql/window/storage/OrderedStoreKey.java
deleted file mode 100644
index e56d3b3..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/window/storage/OrderedStoreKey.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.window.storage;
-
-/**
- * This defines the base class for all keys used in window operators
- */
-public abstract class OrderedStoreKey implements Comparable<OrderedStoreKey> {
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/system/sql/LongOffset.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/system/sql/LongOffset.java b/samza-sql-core/src/main/java/org/apache/samza/system/sql/LongOffset.java
deleted file mode 100644
index 102819d..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/system/sql/LongOffset.java
+++ /dev/null
@@ -1,66 +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.sql;
-
-/**
- * An implementation of {@link org.apache.samza.system.sql.Offset}, w/ {@code long} value as the offset
- */
-public class LongOffset implements Offset {
-
-  /**
-   * The offset value in {@code long}
-   */
-  private final Long offset;
-
-  private LongOffset(long offset) {
-    this.offset = offset;
-  }
-
-  public LongOffset(String offset) {
-    this.offset = Long.valueOf(offset);
-  }
-
-  @Override
-  public int compareTo(Offset o) {
-    if (!(o instanceof LongOffset)) {
-      throw new IllegalArgumentException("Not comparable offset classes. LongOffset vs " + o.getClass().getName());
-    }
-    LongOffset other = (LongOffset) o;
-    return this.offset.compareTo(other.offset);
-  }
-
-  /**
-   * Helper method to get the minimum offset
-   *
-   * @return The minimum offset
-   */
-  public static LongOffset getMinOffset() {
-    return new LongOffset(Long.MIN_VALUE);
-  }
-
-  /**
-   * Helper method to get the maximum offset
-   *
-   * @return The maximum offset
-   */
-  public static LongOffset getMaxOffset() {
-    return new LongOffset(Long.MAX_VALUE);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/system/sql/Offset.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/system/sql/Offset.java b/samza-sql-core/src/main/java/org/apache/samza/system/sql/Offset.java
deleted file mode 100644
index 98547e2..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/system/sql/Offset.java
+++ /dev/null
@@ -1,27 +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.sql;
-
-/**
- * A generic interface extending {@link java.lang.Comparable} to be used as {@code Offset} in a stream
- */
-public interface Offset extends Comparable<Offset> {
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/task/sql/RouterMessageCollector.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/task/sql/RouterMessageCollector.java b/samza-sql-core/src/main/java/org/apache/samza/task/sql/RouterMessageCollector.java
deleted file mode 100644
index 18eb0a3..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/task/sql/RouterMessageCollector.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.task.sql;
-
-import org.apache.samza.sql.api.data.Relation;
-import org.apache.samza.sql.api.data.Tuple;
-import org.apache.samza.sql.api.operators.OperatorRouter;
-import org.apache.samza.sql.api.operators.SimpleOperator;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-
-
-/**
- * This class extends {@link org.apache.samza.task.sql.SimpleMessageCollector} that uses {@link org.apache.samza.sql.api.operators.OperatorRouter}
- *
- */
-public class RouterMessageCollector extends SimpleMessageCollector {
-
-  private final OperatorRouter rteCntx;
-
-  public RouterMessageCollector(MessageCollector collector, TaskCoordinator coordinator, OperatorRouter rteCntx) {
-    super(collector, coordinator);
-    this.rteCntx = rteCntx;
-  }
-
-  @Override
-  protected void realSend(Relation rel) throws Exception {
-    for (SimpleOperator op : this.rteCntx.getNextOperators(rel.getName())) {
-      op.process(rel, this, coordinator);
-    }
-  }
-
-  @Override
-  protected void realSend(Tuple tuple) throws Exception {
-    for (SimpleOperator op : this.rteCntx.getNextOperators(tuple.getEntityName())) {
-      op.process(tuple, this, coordinator);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/task/sql/SimpleMessageCollector.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/task/sql/SimpleMessageCollector.java b/samza-sql-core/src/main/java/org/apache/samza/task/sql/SimpleMessageCollector.java
deleted file mode 100644
index b29838a..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/task/sql/SimpleMessageCollector.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.task.sql;
-
-import org.apache.samza.sql.api.data.Relation;
-import org.apache.samza.sql.api.data.Tuple;
-import org.apache.samza.sql.api.operators.OperatorCallback;
-import org.apache.samza.sql.operators.factory.NoopOperatorCallback;
-import org.apache.samza.storage.kv.Entry;
-import org.apache.samza.storage.kv.KeyValueIterator;
-import org.apache.samza.system.OutgoingMessageEnvelope;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-
-
-public class SimpleMessageCollector implements MessageCollector {
-  protected final MessageCollector collector;
-  protected final TaskCoordinator coordinator;
-  protected OperatorCallback callback;
-
-  /**
-   * Ctor that creates the {@code SimpleMessageCollector} from scratch
-   * @param collector The {@link org.apache.samza.task.MessageCollector} in the context
-   * @param coordinator The {@link org.apache.samza.task.TaskCoordinator} in the context
-   * @param callback The {@link org.apache.samza.sql.api.operators.OperatorCallback} in the context
-   */
-  public SimpleMessageCollector(MessageCollector collector, TaskCoordinator coordinator, OperatorCallback callback) {
-    this.collector = collector;
-    this.coordinator = coordinator;
-    if (callback == null) {
-      this.callback = new NoopOperatorCallback();
-    } else {
-      this.callback = callback;
-    }
-  }
-
-  /**
-   * Ctor that creates the {@code SimpleMessageCollector} from scratch
-   * @param collector The {@link org.apache.samza.task.MessageCollector} in the context
-   * @param coordinator The {@link org.apache.samza.task.TaskCoordinator} in the context
-   */
-  public SimpleMessageCollector(MessageCollector collector, TaskCoordinator coordinator) {
-    this.collector = collector;
-    this.coordinator = coordinator;
-  }
-
-  /**
-   * This method swaps the {@code callback} with the new one
-   *
-   * <p> This method allows the {@link org.apache.samza.sql.api.operators.SimpleOperator} to be swapped when the collector
-   * is passed down into the next operator's context. Hence, under the new operator's context, the correct {@link org.apache.samza.sql.api.operators.OperatorCallback#afterProcess(Relation, MessageCollector, TaskCoordinator)},
-   * and {@link org.apache.samza.sql.api.operators.OperatorCallback#afterProcess(Tuple, MessageCollector, TaskCoordinator)} can be invoked
-   *
-   * @param callback The new {@link org.apache.samza.sql.api.operators.OperatorCallback} to be set
-   */
-  public void switchOperatorCallback(OperatorCallback callback) {
-    this.callback = callback;
-  }
-
-  /**
-   * Method is declared to be final s.t. we enforce that the callback functions are called first
-   */
-  final public void send(Relation deltaRelation) throws Exception {
-    Relation rel = this.callback.afterProcess(deltaRelation, collector, coordinator);
-    if (rel == null) {
-      return;
-    }
-    this.realSend(rel);
-  }
-
-  /**
-   * Method is declared to be final s.t. we enforce that the callback functions are called first
-   */
-  final public void send(Tuple tuple) throws Exception {
-    Tuple otuple = this.callback.afterProcess(tuple, collector, coordinator);
-    if (otuple == null) {
-      return;
-    }
-    this.realSend(otuple);
-  }
-
-  protected void realSend(Relation rel) throws Exception {
-    for (KeyValueIterator<?, Tuple> iter = rel.all(); iter.hasNext();) {
-      Entry<?, Tuple> entry = iter.next();
-      this.collector.send((OutgoingMessageEnvelope) entry.getValue().getMessage());
-    }
-  }
-
-  protected void realSend(Tuple tuple) throws Exception {
-    this.collector.send((OutgoingMessageEnvelope) tuple.getMessage());
-  }
-
-  @Override
-  public void send(OutgoingMessageEnvelope envelope) {
-    this.collector.send(envelope);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/test/java/org/apache/samza/sql/data/serializers/SqlAvroSerdeTest.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/test/java/org/apache/samza/sql/data/serializers/SqlAvroSerdeTest.java b/samza-sql-core/src/test/java/org/apache/samza/sql/data/serializers/SqlAvroSerdeTest.java
deleted file mode 100644
index 7412669..0000000
--- a/samza-sql-core/src/test/java/org/apache/samza/sql/data/serializers/SqlAvroSerdeTest.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.sql.data.serializers;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.generic.GenericDatumWriter;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.io.BinaryEncoder;
-import org.apache.avro.io.DatumWriter;
-import org.apache.avro.io.EncoderFactory;
-import org.apache.samza.config.Config;
-import org.apache.samza.config.MapConfig;
-import org.apache.samza.serializers.Serde;
-import org.apache.samza.sql.data.avro.AvroData;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-public class SqlAvroSerdeTest {
-  public static final String ORDER_SCHEMA = "{\"namespace\": \"org.apache.samza.sql\",\n"+
-      " \"type\": \"record\",\n"+
-      " \"name\": \"Order\",\n"+
-      " \"fields\": [\n"+
-      "     {\"name\": \"id\", \"type\": \"int\"},\n"+
-      "     {\"name\": \"product\",  \"type\": \"string\"},\n"+
-      "     {\"name\": \"quantity\", \"type\": \"int\"}\n"+
-      " ]\n"+
-      "}";
-
-  public static Schema orderSchema = new Schema.Parser().parse(ORDER_SCHEMA);
-  private static Serde serde = new SqlAvroSerdeFactory().getSerde("sqlAvro", sqlAvroSerdeTestConfig());
-
-  @Test
-  public void testSqlAvroSerdeDeserialization() throws IOException {
-    AvroData decodedDatum = (AvroData)serde.fromBytes(encodeMessage(sampleOrderRecord(), orderSchema));
-
-    Assert.assertTrue(decodedDatum.schema().getType() == org.apache.samza.sql.api.data.Schema.Type.STRUCT);
-    Assert.assertTrue(decodedDatum.getFieldData("id").schema().getType() == org.apache.samza.sql.api.data.Schema.Type.INTEGER);
-    Assert.assertTrue(decodedDatum.getFieldData("quantity").schema().getType() == org.apache.samza.sql.api.data.Schema.Type.INTEGER);
-    Assert.assertTrue(decodedDatum.getFieldData("product").schema().getType() == org.apache.samza.sql.api.data.Schema.Type.STRING);
-  }
-
-  @Test
-  public void testSqlAvroSerialization() throws IOException {
-    AvroData decodedDatumOriginal = (AvroData)serde.fromBytes(encodeMessage(sampleOrderRecord(), orderSchema));
-    byte[] encodedDatum = serde.toBytes(decodedDatumOriginal);
-
-    AvroData decodedDatum = (AvroData)serde.fromBytes(encodedDatum);
-
-    Assert.assertTrue(decodedDatum.schema().getType() == org.apache.samza.sql.api.data.Schema.Type.STRUCT);
-    Assert.assertTrue(decodedDatum.getFieldData("id").schema().getType() == org.apache.samza.sql.api.data.Schema.Type.INTEGER);
-    Assert.assertTrue(decodedDatum.getFieldData("quantity").schema().getType() == org.apache.samza.sql.api.data.Schema.Type.INTEGER);
-    Assert.assertTrue(decodedDatum.getFieldData("product").schema().getType() == org.apache.samza.sql.api.data.Schema.Type.STRING);
-  }
-
-  private static Config sqlAvroSerdeTestConfig(){
-    Map<String, String> config = new HashMap<String, String>();
-    config.put("serializers.sqlAvro.schema", ORDER_SCHEMA);
-
-    return new MapConfig(config);
-  }
-
-  private static byte[] encodeMessage(GenericRecord datum, Schema avroSchema) throws IOException {
-    DatumWriter<GenericRecord> writer = new GenericDatumWriter<GenericRecord>(avroSchema);
-    ByteArrayOutputStream output = new ByteArrayOutputStream();
-    BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(output, null);
-    writer.write(datum, encoder);
-    encoder.flush();
-
-    return  output.toByteArray();
-  }
-
-  private static GenericRecord sampleOrderRecord(){
-    GenericData.Record datum = new GenericData.Record(orderSchema);
-    datum.put("id", 1);
-    datum.put("product", "paint");
-    datum.put("quantity", 3);
-
-    return datum;
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/test/java/org/apache/samza/task/sql/RandomWindowOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/test/java/org/apache/samza/task/sql/RandomWindowOperatorTask.java b/samza-sql-core/src/test/java/org/apache/samza/task/sql/RandomWindowOperatorTask.java
deleted file mode 100644
index 20dc701..0000000
--- a/samza-sql-core/src/test/java/org/apache/samza/task/sql/RandomWindowOperatorTask.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.task.sql;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.sql.api.data.Relation;
-import org.apache.samza.sql.api.data.Tuple;
-import org.apache.samza.sql.api.operators.OperatorCallback;
-import org.apache.samza.sql.data.IncomingMessageTuple;
-import org.apache.samza.sql.operators.window.BoundedTimeWindow;
-import org.apache.samza.system.IncomingMessageEnvelope;
-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;
-
-
-/***
- * This example illustrate a use case for the full-state timed window operator
- *
- */
-public class RandomWindowOperatorTask implements StreamTask, InitableTask, WindowableTask {
-  private BoundedTimeWindow wndOp;
-
-  private final OperatorCallback wndCallback = new OperatorCallback() {
-
-    @Override
-    public Tuple beforeProcess(Tuple tuple, MessageCollector collector, TaskCoordinator coordinator) {
-      return tuple;
-    }
-
-    @Override
-    public Relation beforeProcess(Relation rel, MessageCollector collector, TaskCoordinator coordinator) {
-      return rel;
-    }
-
-    @Override
-    public Relation afterProcess(Relation rel, MessageCollector collector, TaskCoordinator coordinator) {
-      return rel;
-    }
-
-    @Override
-    public Tuple afterProcess(Tuple tuple, MessageCollector collector, TaskCoordinator coordinator) {
-      return filterWindowOutput(tuple, collector, coordinator);
-    }
-
-    private Tuple filterWindowOutput(Tuple tuple, MessageCollector collector, TaskCoordinator coordinator) {
-      // filter all delete tuples before send
-      if (tuple.isDelete()) {
-        return null;
-      }
-      return tuple;
-    }
-
-  };
-
-  @Override
-  public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator)
-      throws Exception {
-    // based on tuple's stream name, get the window op and run process()
-    wndOp.process(new IncomingMessageTuple(envelope), collector, coordinator);
-
-  }
-
-  @Override
-  public void window(MessageCollector collector, TaskCoordinator coordinator) throws Exception {
-    // based on tuple's stream name, get the window op and run process()
-    wndOp.refresh(System.nanoTime(), collector, coordinator);
-  }
-
-  @Override
-  public void init(Config config, TaskContext context) throws Exception {
-    // 1. create a fixed length 10 sec window operator
-    this.wndOp = new BoundedTimeWindow("wndOp1", 10, "kafka:stream1", "relation1", this.wndCallback);
-    this.wndOp.init(config, context);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/test/java/org/apache/samza/task/sql/StreamSqlTask.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/test/java/org/apache/samza/task/sql/StreamSqlTask.java b/samza-sql-core/src/test/java/org/apache/samza/task/sql/StreamSqlTask.java
deleted file mode 100644
index 9124e3c..0000000
--- a/samza-sql-core/src/test/java/org/apache/samza/task/sql/StreamSqlTask.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.task.sql;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.sql.data.IncomingMessageTuple;
-import org.apache.samza.sql.operators.factory.SimpleRouter;
-import org.apache.samza.sql.operators.join.StreamStreamJoin;
-import org.apache.samza.sql.operators.partition.PartitionOp;
-import org.apache.samza.sql.operators.window.BoundedTimeWindow;
-import org.apache.samza.system.IncomingMessageEnvelope;
-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;
-
-
-/***
- * This example illustrate a SQL join operation that joins two streams together using the folowing operations:
- * <ul>
- * <li>a. the two streams are each processed by a window operator to convert to relations
- * <li>b. a join operator is applied on the two relations to generate join results
- * <li>c. an istream operator is applied on join output and convert the relation into a stream
- * <li>d. a partition operator that re-partitions the output stream from istream and send the stream to system output
- * </ul>
- *
- * This example also uses an implementation of <code>SqlMessageCollector</code> (@see <code>OperatorMessageCollector</code>)
- * that uses <code>OperatorRouter</code> to automatically execute the whole paths that connects operators together.
- */
-public class StreamSqlTask implements StreamTask, InitableTask, WindowableTask {
-
-  private SimpleRouter rteCntx;
-
-  @Override
-  public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator)
-      throws Exception {
-    this.rteCntx.process(new IncomingMessageTuple(envelope), collector, coordinator);
-  }
-
-  @Override
-  public void window(MessageCollector collector, TaskCoordinator coordinator) throws Exception {
-    this.rteCntx.refresh(System.nanoTime(), collector, coordinator);
-  }
-
-  @Override
-  public void init(Config config, TaskContext context) throws Exception {
-    // create all operators via the operator factory
-    // 1. create two window operators
-    BoundedTimeWindow wnd1 = new BoundedTimeWindow("fixedWnd1", 10, "inputStream1", "fixedWndOutput1");
-    BoundedTimeWindow wnd2 = new BoundedTimeWindow("fixedWnd2", 10, "inputStream2", "fixedWndOutput2");
-    // 2. create one join operator
-    @SuppressWarnings("serial")
-    List<String> inputRelations = new ArrayList<String>() {
-      {
-        add("fixedWndOutput1");
-        add("fixedWndOutput2");
-      }
-    };
-    @SuppressWarnings("serial")
-    List<String> joinKeys = new ArrayList<String>() {
-      {
-        add("key1");
-        add("key2");
-      }
-    };
-    StreamStreamJoin join = new StreamStreamJoin("joinOp", inputRelations, "joinOutput", joinKeys);
-    // 4. create a re-partition operator
-    PartitionOp par = new PartitionOp("parOp1", "joinOutput", "kafka", "parOutputStrm1", "joinKey", 50);
-
-    // Now, connecting the operators via the OperatorRouter
-    this.rteCntx = new SimpleRouter();
-    // 1. set two system input operators (i.e. two window operators)
-    this.rteCntx.addOperator(wnd1);
-    this.rteCntx.addOperator(wnd2);
-    // 2. connect join operator to both window operators
-    this.rteCntx.addOperator(join);
-    // 3. connect re-partition operator to the stream operator
-    this.rteCntx.addOperator(par);
-
-    this.rteCntx.init(config, context);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/test/java/org/apache/samza/task/sql/UserCallbacksSqlTask.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/test/java/org/apache/samza/task/sql/UserCallbacksSqlTask.java b/samza-sql-core/src/test/java/org/apache/samza/task/sql/UserCallbacksSqlTask.java
deleted file mode 100644
index 96e96c3..0000000
--- a/samza-sql-core/src/test/java/org/apache/samza/task/sql/UserCallbacksSqlTask.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.task.sql;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.sql.api.data.Relation;
-import org.apache.samza.sql.api.data.Stream;
-import org.apache.samza.sql.api.data.Tuple;
-import org.apache.samza.sql.api.operators.OperatorCallback;
-import org.apache.samza.sql.data.IncomingMessageTuple;
-import org.apache.samza.sql.operators.factory.SimpleRouter;
-import org.apache.samza.sql.operators.join.StreamStreamJoin;
-import org.apache.samza.sql.operators.partition.PartitionOp;
-import org.apache.samza.sql.operators.window.BoundedTimeWindow;
-import org.apache.samza.system.IncomingMessageEnvelope;
-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;
-
-
-/***
- * This example illustrate a SQL join operation that joins two streams together using the folowing operations:
- * <ul>
- * <li>a. the two streams are each processed by a window operator to convert to relations
- * <li>b. a join operator is applied on the two relations to generate join results
- * <li>c. an istream operator is applied on join output and convert the relation into a stream
- * <li>d. a partition operator that re-partitions the output stream from istream and send the stream to system output
- * </ul>
- *
- * This example also uses an implementation of <code>SqlMessageCollector</code> (@see <code>OperatorMessageCollector</code>)
- * that uses <code>OperatorRouter</code> to automatically execute the whole paths that connects operators together.
- */
-public class UserCallbacksSqlTask implements StreamTask, InitableTask, WindowableTask {
-
-  private SimpleRouter simpleRtr;
-
-  private final OperatorCallback wndCallback = new OperatorCallback() {
-
-    @Override
-    public Tuple beforeProcess(Tuple tuple, MessageCollector collector, TaskCoordinator coordinator) {
-      return filterWindowInput(tuple, collector, coordinator);
-    }
-
-    @Override
-    public Relation beforeProcess(Relation rel, MessageCollector collector, TaskCoordinator coordinator) {
-      return onRelationInput(rel, collector, coordinator);
-    }
-
-    @Override
-    public Relation afterProcess(Relation rel, MessageCollector collector, TaskCoordinator coordinator) {
-      return rel;
-    }
-
-    @Override
-    public Tuple afterProcess(Tuple tuple, MessageCollector collector, TaskCoordinator coordinator) {
-      return tuple;
-    }
-
-    private Tuple filterWindowInput(Tuple tuple, MessageCollector collector, TaskCoordinator coordinator) {
-      // filter all delete tuples before send
-      if (tuple.isDelete()) {
-        return null;
-      }
-      return tuple;
-    }
-
-    private Relation onRelationInput(Relation rel, MessageCollector collector,
-        TaskCoordinator coordinator) {
-      // check whether the input is a stream
-      if (!(rel instanceof Stream<?>)) {
-        throw new IllegalArgumentException("Wrong input entity");
-      }
-      return rel;
-    }
-  };
-
-  @Override
-  public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator)
-      throws Exception {
-    this.simpleRtr.process(new IncomingMessageTuple(envelope), collector, coordinator);
-  }
-
-  @Override
-  public void window(MessageCollector collector, TaskCoordinator coordinator) throws Exception {
-    this.simpleRtr.refresh(System.nanoTime(), collector, coordinator);
-  }
-
-  @Override
-  public void init(Config config, TaskContext context) throws Exception {
-    // create all operators via the operator factory
-    // 1. create two window operators
-    BoundedTimeWindow wnd1 =
-        new BoundedTimeWindow("fixedWnd1", 10, "inputStream1", "fixedWndOutput1", this.wndCallback);
-    BoundedTimeWindow wnd2 =
-        new BoundedTimeWindow("fixedWnd2", 10, "inputStream2", "fixedWndOutput2", this.wndCallback);
-    // 2. create one join operator
-    @SuppressWarnings("serial")
-    List<String> inputRelations = new ArrayList<String>() {
-      {
-        add("fixedWndOutput1");
-        add("fixedWndOutput2");
-      }
-    };
-    @SuppressWarnings("serial")
-    List<String> joinKeys = new ArrayList<String>() {
-      {
-        add("key1");
-        add("key2");
-      }
-    };
-    StreamStreamJoin join = new StreamStreamJoin("joinOp", inputRelations, "joinOutput", joinKeys);
-    // 4. create a re-partition operator
-    PartitionOp par = new PartitionOp("parOp1", "joinOutput", "kafka", "parOutputStrm1", "joinKey", 50);
-
-    // Now, connecting the operators via the OperatorRouter
-    this.simpleRtr = new SimpleRouter();
-    // 1. set two system input operators (i.e. two window operators)
-    this.simpleRtr.addOperator(wnd1);
-    this.simpleRtr.addOperator(wnd2);
-    // 2. connect join operator to both window operators
-    this.simpleRtr.addOperator(join);
-    // 3. connect re-partition operator to the stream operator
-    this.simpleRtr.addOperator(par);
-
-    this.simpleRtr.init(config, context);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/settings.gradle
----------------------------------------------------------------------
diff --git a/settings.gradle b/settings.gradle
index f1903c6..fde8ab5 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -23,7 +23,7 @@ include \
   'samza-shell'
 
 def jdk8Modules = [
-  'samza-sql-core',
+  'samza-operator',
   'samza-sql-calcite'
 ] as HashSet
 


[5/5] samza git commit: SAMZA-914: Creating the fluent programming APIs w/ operators

Posted by ni...@apache.org.
SAMZA-914: Creating the fluent programming APIs w/ operators


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

Branch: refs/heads/samza-sql
Commit: fbdd76daada63e52575815737b9e5c1d4e787e71
Parents: f29c614
Author: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Authored: Tue Oct 4 14:56:14 2016 -0700
Committer: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Committed: Tue Oct 4 14:56:14 2016 -0700

----------------------------------------------------------------------
 build.gradle                                    |   6 +-
 gradle/dependency-versions.gradle               |   4 +-
 .../org/apache/samza/config/TestConfig.java     |   7 +-
 samza-operator/README.md                        |  17 +
 .../samza/operators/api/MessageStream.java      | 162 +++++++
 .../samza/operators/api/MessageStreams.java     |  80 ++++
 .../samza/operators/api/TriggerBuilder.java     | 314 +++++++++++++
 .../apache/samza/operators/api/WindowState.java |  77 +++
 .../org/apache/samza/operators/api/Windows.java | 195 ++++++++
 .../apache/samza/operators/api/data/Data.java   |  57 +++
 .../api/data/IncomingSystemMessage.java         |  76 +++
 .../operators/api/data/InputSystemMessage.java  |  43 ++
 .../samza/operators/api/data/LongOffset.java    |  75 +++
 .../samza/operators/api/data/Message.java       |  58 +++
 .../apache/samza/operators/api/data/Offset.java |  27 ++
 .../apache/samza/operators/api/data/Schema.java |  58 +++
 .../samza/operators/api/internal/Operators.java | 468 +++++++++++++++++++
 .../samza/operators/api/internal/Trigger.java   |  95 ++++
 .../samza/operators/api/internal/WindowFn.java  |  60 +++
 .../operators/api/internal/WindowOutput.java    |  55 +++
 .../samza/operators/impl/ChainedOperators.java  |  73 +++
 .../samza/operators/impl/OperatorImpl.java      |  92 ++++
 .../samza/operators/impl/ProcessorContext.java  |  53 +++
 .../operators/impl/SimpleOperatorImpl.java      |  49 ++
 .../samza/operators/impl/SinkOperatorImpl.java  |  41 ++
 .../samza/operators/impl/StateStoreImpl.java    |  56 +++
 .../operators/impl/data/avro/AvroData.java      | 262 +++++++++++
 .../operators/impl/data/avro/AvroSchema.java    | 296 ++++++++++++
 .../impl/data/serializers/SqlAvroSerde.java     | 108 +++++
 .../data/serializers/SqlAvroSerdeFactory.java   |  40 ++
 .../impl/data/serializers/SqlStringSerde.java   |  44 ++
 .../data/serializers/SqlStringSerdeFactory.java |  33 ++
 .../operators/impl/data/string/StringData.java  | 101 ++++
 .../impl/data/string/StringSchema.java          |  73 +++
 .../impl/window/SessionWindowImpl.java          |  66 +++
 .../samza/task/StreamOperatorAdaptorTask.java   |  85 ++++
 .../apache/samza/task/StreamOperatorTask.java   |  42 ++
 .../apache/samza/operators/api/TestMessage.java |  47 ++
 .../samza/operators/api/TestMessageStreams.java |  35 ++
 .../samza/operators/api/TestTriggerBuilder.java | 211 +++++++++
 .../apache/samza/operators/api/TestWindows.java | 106 +++++
 .../api/data/TestIncomingSystemMessage.java     |  53 +++
 .../operators/api/data/TestLongOffset.java      |  76 +++
 .../operators/api/internal/TestOperators.java   | 129 +++++
 .../operators/api/internal/TestTrigger.java     |  62 +++
 .../api/internal/TestWindowOutput.java          |  36 ++
 .../samza/operators/impl/TestOperatorImpl.java  |  70 +++
 .../samza/operators/impl/TestOutputMessage.java |  47 ++
 .../operators/impl/TestProcessorContext.java    |  40 ++
 .../operators/impl/TestSimpleOperatorImpl.java  |  54 +++
 .../operators/impl/TestSinkOperatorImpl.java    |  45 ++
 .../operators/impl/TestStateStoreImpl.java      |  69 +++
 .../impl/data/serializers/SqlAvroSerdeTest.java | 102 ++++
 .../impl/window/TestSessionWindowImpl.java      | 132 ++++++
 .../samza/task/BroadcastOperatorTask.java       | 110 +++++
 .../samza/task/InputJsonSystemMessage.java      |  63 +++
 .../org/apache/samza/task/JoinOperatorTask.java |  79 ++++
 .../task/TestStreamOperatorAdaptorTask.java     |  79 ++++
 .../samza/task/TestStreamOperatorTasks.java     | 105 +++++
 .../apache/samza/task/WindowOperatorTask.java   |  70 +++
 .../calcite/schema/TestAvroSchemaConverter.java |   3 +-
 samza-sql-core/README.md                        |  17 -
 .../org/apache/samza/sql/api/data/Data.java     |  54 ---
 .../apache/samza/sql/api/data/EntityName.java   | 140 ------
 .../org/apache/samza/sql/api/data/Relation.java |  40 --
 .../org/apache/samza/sql/api/data/Schema.java   |  55 ---
 .../org/apache/samza/sql/api/data/Stream.java   |  40 --
 .../org/apache/samza/sql/api/data/Table.java    |  38 --
 .../org/apache/samza/sql/api/data/Tuple.java    |  75 ---
 .../samza/sql/api/operators/Operator.java       |  73 ---
 .../sql/api/operators/OperatorCallback.java     |  70 ---
 .../samza/sql/api/operators/OperatorRouter.java |  54 ---
 .../samza/sql/api/operators/OperatorSpec.java   |  58 ---
 .../samza/sql/api/operators/SimpleOperator.java |  34 --
 .../sql/api/operators/SqlOperatorFactory.java   |  37 --
 .../samza/sql/data/IncomingMessageTuple.java    |  94 ----
 .../apache/samza/sql/data/avro/AvroData.java    | 262 -----------
 .../apache/samza/sql/data/avro/AvroSchema.java  | 296 ------------
 .../sql/data/serializers/SqlAvroSerde.java      | 109 -----
 .../data/serializers/SqlAvroSerdeFactory.java   |  40 --
 .../sql/data/serializers/SqlStringSerde.java    |  45 --
 .../data/serializers/SqlStringSerdeFactory.java |  33 --
 .../samza/sql/data/string/StringData.java       | 101 ----
 .../samza/sql/data/string/StringSchema.java     |  73 ---
 .../operators/factory/NoopOperatorCallback.java |  50 --
 .../factory/SimpleOperatorFactoryImpl.java      |  51 --
 .../operators/factory/SimpleOperatorImpl.java   | 136 ------
 .../operators/factory/SimpleOperatorSpec.java   | 106 -----
 .../sql/operators/factory/SimpleRouter.java     | 136 ------
 .../sql/operators/join/StreamStreamJoin.java    | 117 -----
 .../operators/join/StreamStreamJoinSpec.java    |  38 --
 .../sql/operators/partition/PartitionOp.java    | 120 -----
 .../sql/operators/partition/PartitionSpec.java  |  91 ----
 .../sql/operators/window/BoundedTimeWindow.java | 161 -------
 .../samza/sql/operators/window/WindowSpec.java  |  67 ---
 .../samza/sql/operators/window/WindowState.java |  44 --
 .../sql/window/storage/OrderedStoreKey.java     |  26 --
 .../org/apache/samza/system/sql/LongOffset.java |  66 ---
 .../org/apache/samza/system/sql/Offset.java     |  27 --
 .../samza/task/sql/RouterMessageCollector.java  |  56 ---
 .../samza/task/sql/SimpleMessageCollector.java  | 114 -----
 .../sql/data/serializers/SqlAvroSerdeTest.java  | 102 ----
 .../task/sql/RandomWindowOperatorTask.java      |  96 ----
 .../apache/samza/task/sql/StreamSqlTask.java    | 104 -----
 .../samza/task/sql/UserCallbacksSqlTask.java    | 150 ------
 settings.gradle                                 |   2 +-
 106 files changed, 5265 insertions(+), 3704 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 8fe4c46..aeefd1c 100644
--- a/build.gradle
+++ b/build.gradle
@@ -336,7 +336,7 @@ project(":samza-yarn_$scalaVersion") {
 }
 
 if (JavaVersion.current().isJava8Compatible()) {
-  project(":samza-sql-core") {
+  project(":samza-operator") {
     apply plugin: 'java'
     
     sourceCompatibility = 1.8
@@ -345,7 +345,9 @@ if (JavaVersion.current().isJava8Compatible()) {
       compile project(':samza-api')
       compile project(":samza-core_$scalaVersion")
       compile "commons-collections:commons-collections:$commonsCollectionVersion"
+      compile "org.apache.commons:commons-lang3:$commonsLang3Version"
       compile "org.apache.avro:avro:$avroVersion"
+      compile "org.reactivestreams:reactive-streams:$reactiveStreamVersion"
       testCompile "junit:junit:$junitVersion"
       testCompile "org.mockito:mockito-all:$mockitoVersion"
     }
@@ -363,7 +365,7 @@ if (JavaVersion.current().isJava8Compatible()) {
     sourceCompatibility = 1.8
 
     dependencies {
-      compile project(":samza-sql-core")
+      compile project(":samza-operator")
       compile "org.apache.calcite:calcite-core:$calciteVersion"
       testCompile "junit:junit:$junitVersion"
       testCompile "org.mockito:mockito-all:$mockitoVersion"

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/gradle/dependency-versions.gradle
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle
index c52960c..845a3fd 100644
--- a/gradle/dependency-versions.gradle
+++ b/gradle/dependency-versions.gradle
@@ -36,7 +36,9 @@
   guavaVersion = "17.0"
   commonsCodecVersion = "1.9"
   commonsCollectionVersion = "3.2.1"
-  avroVersion = "1.7.7"
+  avroVersion = "1.4.0"
   calciteVersion = "1.2.0-incubating"
   httpClientVersion="4.4.1"
+  reactiveStreamVersion="1.0.0"
+  commonsLang3Version="3.4"
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-api/src/test/java/org/apache/samza/config/TestConfig.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/config/TestConfig.java b/samza-api/src/test/java/org/apache/samza/config/TestConfig.java
index 5d066c5..7d9d56c 100644
--- a/samza-api/src/test/java/org/apache/samza/config/TestConfig.java
+++ b/samza-api/src/test/java/org/apache/samza/config/TestConfig.java
@@ -19,12 +19,13 @@
 
 package org.apache.samza.config;
 
-import static org.junit.Assert.*;
+import org.junit.Test;
 
-import java.util.Map;
 import java.util.HashMap;
+import java.util.Map;
 
-import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 
 public class TestConfig {
   /**

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/README.md
----------------------------------------------------------------------
diff --git a/samza-operator/README.md b/samza-operator/README.md
new file mode 100644
index 0000000..15d2092
--- /dev/null
+++ b/samza-operator/README.md
@@ -0,0 +1,17 @@
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+samza-operator is an experimental module that is under development (SAMZA-552).

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java b/samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
new file mode 100644
index 0000000..a01cee9
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.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.operators.api;
+
+import org.apache.samza.operators.api.data.Message;
+import org.apache.samza.operators.api.internal.WindowOutput;
+import org.apache.samza.operators.api.internal.Operators;
+import org.apache.samza.operators.api.Windows.Window;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+
+/**
+ * This class defines either the input or output streams to/from the operators. Users use the API methods defined here to
+ * directly program the stream processing stages that processes a stream and generate another one.
+ *
+ * @param <M>  Type of message in this stream
+ */
+public class MessageStream<M extends Message> {
+
+  /**
+   * Public API methods start here
+   */
+
+  /**
+   * Defines a function API that takes three input parameters w/ types {@code A}, {@code B}, and {@code C} and w/o a return value
+   *
+   * @param <A>  the type of input {@code a}
+   * @param <B>  the type of input {@code b}
+   * @param <C>  the type of input {@code c}
+   */
+  @FunctionalInterface
+  public interface VoidFunction3 <A, B, C> {
+    public void apply(A a, B b, C c);
+  }
+
+  /**
+   * Method to apply a map function (1:1) on a {@link MessageStream}
+   *
+   * @param mapper  the mapper function to map one input {@link Message} to one output {@link Message}
+   * @param <OM>  the type of the output {@link Message} in the output {@link MessageStream}
+   * @return the output {@link MessageStream} by applying the map function on the input {@link MessageStream}
+   */
+  public <OM extends Message> MessageStream<OM> map(Function<M, OM> mapper) {
+    return Operators.<M, OM>getStreamOperator(m -> new ArrayList<OM>() {{
+      OM r = mapper.apply(m);
+      if (r != null) {
+        this.add(r);
+      }
+    }}).getOutputStream();
+  }
+
+  /**
+   * Method to apply a flatMap function (1:n) on a {@link MessageStream}
+   *
+   * @param flatMapper  the flat mapper function to map one input {@link Message} to zero or more output {@link Message}s
+   * @param <OM>  the type of the output {@link Message} in the output {@link MessageStream}
+   * @return the output {@link MessageStream} by applying the map function on the input {@link MessageStream}
+   */
+  public <OM extends Message> MessageStream<OM> flatMap(Function<M, Collection<OM>> flatMapper) {
+    return Operators.getStreamOperator(flatMapper).getOutputStream();
+  }
+
+  /**
+   * Method to apply a filter function on a {@link MessageStream}
+   *
+   * @param filter  the filter function to filter input {@link Message}s from the input {@link MessageStream}
+   * @return the output {@link MessageStream} after applying the filter function on the input {@link MessageStream}
+   */
+  public MessageStream<M> filter(Function<M, Boolean> filter) {
+    return Operators.<M, M>getStreamOperator(t -> new ArrayList<M>() {{
+      if (filter.apply(t)) {
+        this.add(t);
+      }
+    }}).getOutputStream();
+  }
+
+  /**
+   * Method to send an input {@link MessageStream} to an output {@link SystemStream}, and allows the output {@link MessageStream}
+   * to be consumed by downstream stream operators again.
+   *
+   * @param sink  the user-defined sink function to send the input {@link Message}s to the external output systems
+   */
+  public void sink(VoidFunction3<M, MessageCollector, TaskCoordinator> sink) {
+    Operators.getSinkOperator(sink);
+  }
+
+  /**
+   * Method to perform a window function (i.e. a group-by, aggregate function) on a {@link MessageStream}
+   *
+   * @param window  the window function to group and aggregate the input {@link Message}s from the input {@link MessageStream}
+   * @param <WK>  the type of key in the output {@link Message} from the {@link Window} function
+   * @param <WV>  the type of output value from
+   * @param <WS>  the type of window state kept in the {@link Window} function
+   * @param <WM>  the type of {@link WindowOutput} message from the {@link Window} function
+   * @return the output {@link MessageStream} after applying the window function on the input {@link MessageStream}
+   */
+  public <WK, WV, WS extends WindowState<WV>, WM extends WindowOutput<WK, WV>> MessageStream<WM> window(Window<M, WK, WV, WM> window) {
+    return Operators.getWindowOperator(Windows.getInternalWindowFn(window)).getOutputStream();
+  }
+
+  /**
+   * Method to add an input {@link MessageStream} to a join function. Note that we currently only support 2-way joins.
+   *
+   * @param other  the other stream to be joined w/
+   * @param merger  the common function to merge messages from this {@link MessageStream} and {@code other}
+   * @param <K>  the type of join key
+   * @param <JM>  the type of message in the {@link Message} from the other join stream
+   * @param <RM>  the type of message in the {@link Message} from the join function
+   * @return the output {@link MessageStream} from the join function {@code joiner}
+   */
+  public <K, JM extends Message<K, ?>, RM extends Message> MessageStream<RM> join(MessageStream<JM> other,
+      BiFunction<M, JM, RM> merger) {
+    MessageStream<RM> outputStream = new MessageStream<>();
+
+    BiFunction<M, JM, RM> parJoin1 = merger::apply;
+    BiFunction<JM, M, RM> parJoin2 = (m, t1) -> merger.apply(t1, m);
+
+    // TODO: need to add default store functions for the two partial join functions
+
+    Operators.<JM, K, M, RM>getPartialJoinOperator(parJoin2, outputStream);
+    Operators.<M, K, JM, RM>getPartialJoinOperator(parJoin1, outputStream);
+    return outputStream;
+  }
+
+  /**
+   * Method to merge all {@code others} streams w/ this {@link MessageStream}. The merging streams must have the same type {@code M}
+   *
+   * @param others  other streams to be merged w/ this one
+   * @return  the merged output stream
+   */
+  public MessageStream<M> merge(Collection<MessageStream<M>> others) {
+    MessageStream<M> outputStream = new MessageStream<>();
+
+    others.add(this);
+    others.forEach(other -> Operators.getMergeOperator(outputStream));
+    return outputStream;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java b/samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
new file mode 100644
index 0000000..59dd91c
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.api;
+
+import org.apache.samza.operators.api.data.IncomingSystemMessage;
+import org.apache.samza.system.SystemStreamPartition;
+
+
+/**
+ * This class defines all methods to create a {@link MessageStream} object. Users can use this to create an {@link MessageStream}
+ * from a specific input source.
+ *
+ */
+
+public final class MessageStreams {
+
+  /**
+   * private constructor to prevent instantiation
+   */
+  private MessageStreams() {}
+
+  /**
+   * private class for system input/output {@link MessageStream}
+   */
+  public static final class SystemMessageStream extends MessageStream<IncomingSystemMessage> {
+    /**
+     * The corresponding {@link org.apache.samza.system.SystemStream}
+     */
+    private final SystemStreamPartition ssp;
+
+    /**
+     * Constructor for input system stream
+     *
+     * @param ssp  the input {@link SystemStreamPartition} for the input {@link SystemMessageStream}
+     */
+    private SystemMessageStream(SystemStreamPartition ssp) {
+      this.ssp = ssp;
+    }
+
+    /**
+     * Getter for the {@link SystemStreamPartition} of the input
+     *
+     * @return the input {@link SystemStreamPartition}
+     */
+    public SystemStreamPartition getSystemStreamPartition() {
+      return this.ssp;
+    }
+  }
+
+  /**
+   * Public static API methods start here
+   */
+
+  /**
+   * Static API method to create a {@link MessageStream} from a system input stream
+   *
+   * @param ssp  the input {@link SystemStreamPartition}
+   * @return the {@link MessageStream} object takes {@code ssp} as the input
+   */
+  public static SystemMessageStream input(SystemStreamPartition ssp) {
+    return new SystemMessageStream(ssp);
+  }
+
+}

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

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java b/samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java
new file mode 100644
index 0000000..402cc42
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.api;
+
+/**
+ * This interface defines the methods a window state class has to implement. The programmers are allowed to implement
+ * customized window state to be stored in window state stores by implementing this interface class.
+ *
+ * @param <WV>  the type for window output value
+ */
+public interface WindowState<WV> {
+  /**
+   * Method to get the system time when the first message in the window is received
+   *
+   * @return  nano-second of system time for the first message received in the window
+   */
+  long getFirstMessageTimeNs();
+
+  /**
+   * Method to get the system time when the last message in the window is received
+   *
+   * @return  nano-second of system time for the last message received in the window
+   */
+  long getLastMessageTimeNs();
+
+  /**
+   * Method to get the earliest event time in the window
+   *
+   * @return  the earliest event time in nano-second in the window
+   */
+  long getEarliestEventTimeNs();
+
+  /**
+   * Method to get the latest event time in the window
+   *
+   * @return  the latest event time in nano-second in the window
+   */
+  long getLatestEventTimeNs();
+
+  /**
+   * Method to get the total number of messages received in the window
+   *
+   * @return  number of messages in the window
+   */
+  long getNumberMessages();
+
+  /**
+   * Method to get the corresponding window's output value
+   *
+   * @return  the corresponding window's output value
+   */
+  WV getOutputValue();
+
+  /**
+   * Method to set the corresponding window's output value
+   *
+   * @param value  the corresponding window's output value
+   */
+  void setOutputValue(WV value);
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java b/samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java
new file mode 100644
index 0000000..e557b34
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.api;
+
+import org.apache.samza.operators.api.data.Message;
+import org.apache.samza.operators.api.internal.WindowOutput;
+import org.apache.samza.operators.api.internal.Trigger;
+import org.apache.samza.operators.api.internal.Operators;
+import org.apache.samza.operators.api.internal.WindowFn;
+import org.apache.samza.storage.kv.Entry;
+
+import java.util.Collection;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+
+/**
+ * This class defines a collection of {@link Window} functions. The public classes and methods here are intended to be
+ * used by the user (i.e. programmers) to create {@link Window} function directly.
+ *
+ */
+public final class Windows {
+
+  /**
+   * private constructor to prevent instantiation
+   */
+  private Windows() {}
+
+  /**
+   * This class defines a session window function class
+   *
+   * @param <M>  the type of input {@link Message}
+   * @param <WK>  the type of session key in the session window
+   * @param <WV>  the type of output value in each session window
+   */
+  static class SessionWindow<M extends Message, WK, WV> implements Window<M, WK, WV, WindowOutput<WK, WV>> {
+
+    /**
+     * Constructor. Made private s.t. it can only be instantiated via the static API methods in {@link Windows}
+     *
+     * @param sessionKeyFunction  function to get the session key from the input {@link Message}
+     * @param aggregator  function to calculate the output value based on the input {@link Message} and current output value
+     */
+    private SessionWindow(Function<M, WK> sessionKeyFunction, BiFunction<M, WV, WV> aggregator) {
+      this.wndKeyFunction = sessionKeyFunction;
+      this.aggregator = aggregator;
+    }
+
+    /**
+     * function to calculate the window key from input message
+     */
+    private final Function<M, WK> wndKeyFunction;
+
+    /**
+     * function to calculate the output value from the input message and the current output value
+     */
+    private final BiFunction<M, WV, WV> aggregator;
+
+    /**
+     * trigger condition that determines when to send out the output value in a {@link WindowOutput} message
+     */
+    private Trigger<M, WindowState<WV>> trigger = null;
+
+    //TODO: need to create a set of {@link StoreFunctions} that is default to input {@link Message} type for {@link Window}
+    private Operators.StoreFunctions<M, WK, WindowState<WV>> storeFunctions = null;
+
+    /**
+     * Public API methods start here
+     */
+
+    /**
+     * Public API method to define the watermark trigger for the window operator
+     *
+     * @param wndTrigger {@link Trigger} function defines the watermark trigger for this {@link SessionWindow}
+     * @return The window operator w/ the defined watermark trigger
+     */
+    @Override
+    public Window<M, WK, WV, WindowOutput<WK, WV>> setTriggers(TriggerBuilder<M, WV> wndTrigger) {
+      this.trigger = wndTrigger.build();
+      return this;
+    }
+
+    private BiFunction<M, Entry<WK, WindowState<WV>>, WindowOutput<WK, WV>> getTransformFunc() {
+      // TODO: actual implementation of the main session window logic, based on the wndKeyFunction, aggregator, and triggers;
+      return null;
+    }
+
+    private WindowFn<M, WK, WindowState<WV>, WindowOutput<WK, WV>> getInternalWindowFn() {
+      return new WindowFn<M, WK, WindowState<WV>, WindowOutput<WK, WV>>() {
+
+        @Override public BiFunction<M, Entry<WK, WindowState<WV>>, WindowOutput<WK, WV>> getTransformFunc() {
+          return SessionWindow.this.getTransformFunc();
+        }
+
+        @Override public Operators.StoreFunctions<M, WK, WindowState<WV>> getStoreFuncs() {
+          return SessionWindow.this.storeFunctions;
+        }
+
+        @Override public Trigger<M, WindowState<WV>> getTrigger() {
+          return SessionWindow.this.trigger;
+        }
+      };
+    }
+  }
+
+  static <M extends Message, WK, WV, WS extends WindowState<WV>, WM extends WindowOutput<WK, WV>> WindowFn<M, WK, WS, WM> getInternalWindowFn(
+      Window<M, WK, WV, WM> window) {
+    if (window instanceof SessionWindow) {
+      SessionWindow<M, WK, WV> sessionWindow = (SessionWindow<M, WK, WV>) window;
+      return (WindowFn<M, WK, WS, WM>) sessionWindow.getInternalWindowFn();
+    }
+    throw new IllegalArgumentException("Input window type not supported.");
+  }
+
+  /**
+   * Public static API methods start here
+   *
+   */
+
+  /**
+   * The public programming interface class for window function
+   *
+   * @param <M>  the type of input {@link Message}
+   * @param <WK>  the type of key to the {@link Window}
+   * @param <WV>  the type of output value in the {@link WindowOutput}
+   * @param <WM>  the type of message in the window output stream
+   */
+  public interface Window<M extends Message, WK, WV, WM extends WindowOutput<WK, WV>> {
+
+    /**
+     * Set the triggers for this {@link Window}
+     *
+     * @param wndTrigger  trigger conditions set by the programmers
+     * @return  the {@link Window} function w/ the trigger {@code wndTrigger}
+     */
+    Window<M, WK, WV, WM> setTriggers(TriggerBuilder<M, WV> wndTrigger);
+  }
+
+  /**
+   * Static API method to create a {@link SessionWindow} in which the output value is simply the collection of input messages
+   *
+   * @param sessionKeyFunction  function to calculate session window key
+   * @param <M>  type of input {@link Message}
+   * @param <WK>  type of the session window key
+   * @return  the {@link Window} function for the session
+   */
+  public static <M extends Message, WK> Window<M, WK, Collection<M>, WindowOutput<WK, Collection<M>>> intoSessions(Function<M, WK> sessionKeyFunction) {
+    return new SessionWindow<>(sessionKeyFunction, (m, c) -> { c.add(m); return c; });
+  }
+
+  /**
+   * Static API method to create a {@link SessionWindow} in which the output value is a collection of {@code SI} from the input messages
+   *
+   * @param sessionKeyFunction  function to calculate session window key
+   * @param sessionInfoExtractor  function to retrieve session info of type {@code SI} from the input message of type {@code M}
+   * @param <M>  type of the input {@link Message}
+   * @param <WK>  type of the session window key
+   * @param <SI>  type of the session information retrieved from each input message of type {@code M}
+   * @return  the {@link Window} function for the session
+   */
+  public static <M extends Message, WK, SI> Window<M, WK, Collection<SI>, WindowOutput<WK, Collection<SI>>> intoSessions(Function<M, WK> sessionKeyFunction,
+      Function<M, SI> sessionInfoExtractor) {
+    return new SessionWindow<>(sessionKeyFunction,
+        (m, c) -> { c.add(sessionInfoExtractor.apply(m)); return c; } );
+  }
+
+  /**
+   * Static API method to create a {@link SessionWindow} as a counter of input messages
+   *
+   * @param sessionKeyFunction  function to calculate session window key
+   * @param <M>  type of the input {@link Message}
+   * @param <WK>  type of the session window key
+   * @return  the {@link Window} function for the session
+   */
+  public static <M extends Message, WK> Window<M, WK, Integer, WindowOutput<WK, Integer>> intoSessionCounter(Function<M, WK> sessionKeyFunction) {
+    return new SessionWindow<>(sessionKeyFunction, (m, c) -> c + 1);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/api/data/Data.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/api/data/Data.java b/samza-operator/src/main/java/org/apache/samza/operators/api/data/Data.java
new file mode 100644
index 0000000..69a3bee
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/api/data/Data.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.operators.api.data;
+
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * A generic data interface that allows to implement data access / deserialization w/ {@link Schema}
+ */
+public interface Data {
+
+  Schema schema();
+
+  Object value();
+
+  int intValue();
+
+  long longValue();
+
+  float floatValue();
+
+  double doubleValue();
+
+  boolean booleanValue();
+
+  String strValue();
+
+  byte[] bytesValue();
+
+  List<Object> arrayValue();
+
+  Map<Object, Object> mapValue();
+
+  Data getElement(int index);
+
+  Data getFieldData(String fldName);
+
+}

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

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/api/data/InputSystemMessage.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/api/data/InputSystemMessage.java b/samza-operator/src/main/java/org/apache/samza/operators/api/data/InputSystemMessage.java
new file mode 100644
index 0000000..c786025
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/api/data/InputSystemMessage.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.api.data;
+
+import org.apache.samza.system.SystemStreamPartition;
+
+
+/**
+ * This interface defines additional methods a message from an system input should implement, including the methods to
+ * get {@link SystemStreamPartition} and the {@link Offset} of the input system message.
+ */
+public interface InputSystemMessage<O extends Offset> {
+
+  /**
+   * Get the input message's {@link SystemStreamPartition}
+   *
+   * @return  the {@link SystemStreamPartition} this message is coming from
+   */
+  SystemStreamPartition getSystemStreamPartition();
+
+  /**
+   * Get the offset of the message in the input stream. This should be used to uniquely identify a message in an input stream.
+   *
+   * @return The offset of the message in the input stream.
+   */
+  O getOffset();
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/api/data/LongOffset.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/api/data/LongOffset.java b/samza-operator/src/main/java/org/apache/samza/operators/api/data/LongOffset.java
new file mode 100644
index 0000000..f059b33
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/api/data/LongOffset.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.operators.api.data;
+
+/**
+ * An implementation of {@link org.apache.samza.operators.api.data.Offset}, w/ {@code long} value as the offset
+ */
+public class LongOffset implements Offset {
+
+  /**
+   * The offset value in {@code long}
+   */
+  private final Long offset;
+
+  private LongOffset(long offset) {
+    this.offset = offset;
+  }
+
+  public LongOffset(String offset) {
+    this.offset = Long.valueOf(offset);
+  }
+
+  @Override
+  public int compareTo(Offset o) {
+    if (!(o instanceof LongOffset)) {
+      throw new IllegalArgumentException("Not comparable offset classes. LongOffset vs " + o.getClass().getName());
+    }
+    LongOffset other = (LongOffset) o;
+    return this.offset.compareTo(other.offset);
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof LongOffset)) {
+      return false;
+    }
+    LongOffset o = (LongOffset) other;
+    return this.offset.equals(o.offset);
+  }
+
+  /**
+   * Helper method to get the minimum offset
+   *
+   * @return The minimum offset
+   */
+  public static LongOffset getMinOffset() {
+    return new LongOffset(Long.MIN_VALUE);
+  }
+
+  /**
+   * Helper method to get the maximum offset
+   *
+   * @return The maximum offset
+   */
+  public static LongOffset getMaxOffset() {
+    return new LongOffset(Long.MAX_VALUE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java b/samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java
new file mode 100644
index 0000000..9b53b45
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.operators.api.data;
+
+/**
+ * This class defines the generic interface of {@link Message}, which is a entry in the input/output stream.
+ *
+ * <p>The {@link Message} models the basic operatible unit in streaming SQL processes in Samza.
+ *
+ */
+public interface Message<K, M> {
+
+  /**
+   * Access method to get the corresponding message body in {@link Message}
+   *
+   * @return Message object in this {@link Message}
+   */
+  M getMessage();
+
+  /**
+   * Method to indicate whether this {@link Message} indicates deletion of a message w/ the message key
+   *
+   * @return A boolean value indicates whether the current message is a delete or insert message
+   */
+  default boolean isDelete() { return false; };
+
+  /**
+   * Access method to the key of the message
+   *
+   * @return The key of the message
+   */
+  K getKey();
+
+  /**
+   * Get the message creation timestamp of the message.
+   *
+   * @return The message's timestamp in nano seconds.
+   */
+  long getTimestamp();
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/api/data/Offset.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/api/data/Offset.java b/samza-operator/src/main/java/org/apache/samza/operators/api/data/Offset.java
new file mode 100644
index 0000000..0fac2c0
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/api/data/Offset.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.operators.api.data;
+
+/**
+ * A generic interface extending {@link java.lang.Comparable} to be used as {@code Offset} in a stream
+ */
+public interface Offset extends Comparable<Offset> {
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/api/data/Schema.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/api/data/Schema.java b/samza-operator/src/main/java/org/apache/samza/operators/api/data/Schema.java
new file mode 100644
index 0000000..dc3f8f4
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/api/data/Schema.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.operators.api.data;
+
+import java.util.Map;
+
+
+/**
+ * This defines an interface for generic schema access methods
+ */
+public interface Schema {
+
+  enum Type {
+    INTEGER,
+    LONG,
+    FLOAT,
+    DOUBLE,
+    BOOLEAN,
+    STRING,
+    BYTES,
+    STRUCT,
+    ARRAY,
+    MAP
+  };
+
+  Type getType();
+
+  Schema getElementType();
+
+  Schema getValueType();
+
+  Map<String, Schema> getFields();
+
+  Schema getFieldType(String fldName);
+
+  Data read(Object object);
+
+  Data transform(Data inputData);
+
+  boolean equals(Schema other);
+}

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

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java b/samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
new file mode 100644
index 0000000..33a0134
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.api.internal;
+
+import org.apache.samza.operators.api.WindowState;
+import org.apache.samza.operators.api.data.Message;
+
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+/**
+ * Defines the trigger functions for {@link Operators.WindowOperator}. This class is immutable.
+ *
+ * @param <M>  the type of message from the input stream
+ * @param <S>  the type of state variable in the window's state store
+ */
+public class Trigger<M extends Message, S extends WindowState> {
+
+  /**
+   * System timer based trigger condition. This is the only guarantee that the {@link Operators.WindowOperator} will proceed forward
+   */
+  private final Function<S, Boolean> timerTrigger;
+
+  /**
+   * early trigger condition that determines when to send the first output from the {@link Operators.WindowOperator}
+   */
+  private final BiFunction<M, S, Boolean> earlyTrigger;
+
+  /**
+   * late trigger condition that determines when to send the updated output after the first one from a {@link Operators.WindowOperator}
+   */
+  private final BiFunction<M, S, Boolean> lateTrigger;
+
+  /**
+   * the function to updated the window state when the first output is triggered
+   */
+  private final Function<S, S> earlyTriggerUpdater;
+
+  /**
+   * the function to updated the window state when the late output is triggered
+   */
+  private final Function<S, S> lateTriggerUpdater;
+
+  /**
+   * Private constructor to prevent instantiation
+   *
+   * @param timerTrigger  system timer trigger condition
+   * @param earlyTrigger  early trigger condition
+   * @param lateTrigger   late trigger condition
+   * @param earlyTriggerUpdater  early trigger state updater
+   * @param lateTriggerUpdater   late trigger state updater
+   */
+  private Trigger(Function<S, Boolean> timerTrigger, BiFunction<M, S, Boolean> earlyTrigger, BiFunction<M, S, Boolean> lateTrigger,
+      Function<S, S> earlyTriggerUpdater, Function<S, S> lateTriggerUpdater) {
+    this.timerTrigger = timerTrigger;
+    this.earlyTrigger = earlyTrigger;
+    this.lateTrigger = lateTrigger;
+    this.earlyTriggerUpdater = earlyTriggerUpdater;
+    this.lateTriggerUpdater = lateTriggerUpdater;
+  }
+
+  /**
+   * Static method to create a {@link Trigger} object
+   *
+   * @param timerTrigger  system timer trigger condition
+   * @param earlyTrigger  early trigger condition
+   * @param lateTrigger  late trigger condition
+   * @param earlyTriggerUpdater  early trigger state updater
+   * @param lateTriggerUpdater  late trigger state updater
+   * @param <M>  the type of input {@link Message}
+   * @param <S>  the type of window state extends {@link WindowState}
+   * @return  the {@link Trigger} function
+   */
+  public static <M extends Message, S extends WindowState> Trigger<M, S> createTrigger(Function<S, Boolean> timerTrigger,
+      BiFunction<M, S, Boolean> earlyTrigger, BiFunction<M, S, Boolean> lateTrigger, Function<S, S> earlyTriggerUpdater,
+      Function<S, S> lateTriggerUpdater) {
+    return new Trigger(timerTrigger, earlyTrigger, lateTrigger, earlyTriggerUpdater, lateTriggerUpdater);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowFn.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowFn.java b/samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowFn.java
new file mode 100644
index 0000000..1fd88e7
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowFn.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.api.internal;
+
+import org.apache.samza.operators.api.WindowState;
+import org.apache.samza.operators.api.data.Message;
+import org.apache.samza.storage.kv.Entry;
+
+import java.util.function.BiFunction;
+
+
+/**
+ * Defines an internal representation of a window function. This class SHOULD NOT be used by the programmer directly. It is used
+ * by the internal representation and implementation classes in operators.
+ *
+ * @param <M> type of input stream {@link Message} for window
+ * @param <WK>  type of window key in the output {@link Message}
+ * @param <WS>  type of {@link WindowState} variable in the state store
+ * @param <WM>  type of the message in the output stream
+ */
+public interface WindowFn<M extends Message, WK, WS extends WindowState, WM extends WindowOutput<WK, ?>> {
+
+  /**
+   * get the transformation function of the {@link WindowFn}
+   *
+   * @return  the transformation function takes type {@code M} message and the window state entry, then transform to an {@link WindowOutput}
+   */
+  BiFunction<M, Entry<WK, WS>, WM> getTransformFunc();
+
+  /**
+   * get the state store functions for this {@link WindowFn}
+   *
+   * @return  the collection of state store methods
+   */
+  Operators.StoreFunctions<M, WK, WS> getStoreFuncs();
+
+  /**
+   * get the trigger conditions for this {@link WindowFn}
+   *
+   * @return  the trigger condition for the {@link WindowFn} function
+   */
+  Trigger<M, WS> getTrigger();
+
+}


[4/5] samza git commit: SAMZA-914: Creating the fluent programming APIs w/ operators

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowOutput.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowOutput.java b/samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowOutput.java
new file mode 100644
index 0000000..e202c20
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowOutput.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.api.internal;
+
+import org.apache.samza.operators.api.data.Message;
+
+
+/**
+ * This class defines the specific type of output messages from a {@link Operators.WindowOperator} function
+ *
+ * @param <K>  the type of key in the output window result
+ * @param <M>  the type of value in the output window result
+ */
+public final class WindowOutput<K, M> implements Message<K, M> {
+  private final K key;
+  private final M value;
+
+  WindowOutput(K key, M aggregated) {
+    this.key = key;
+    this.value = aggregated;
+  }
+
+  @Override public M getMessage() {
+    return this.value;
+  }
+
+  @Override public K getKey() {
+    return this.key;
+  }
+
+  @Override public long getTimestamp() {
+    return 0;
+  }
+
+  static public <K, M> WindowOutput<K, M> of(K key, M result) {
+    return new WindowOutput<>(key, result);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
new file mode 100644
index 0000000..49cfdeb
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.operators.impl;
+
+import org.apache.samza.operators.api.MessageStream;
+import org.apache.samza.operators.api.data.Message;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+
+
+/**
+ * Implementation class for a chain of operators from the single input {@code source}
+ *
+ * @param <M>  type of message in the input stream {@code source}
+ */
+public class ChainedOperators<M extends Message> {
+
+  /**
+   * Private constructor
+   *
+   * @param source  the input source {@link MessageStream}
+   * @param context  the {@link TaskContext} object that we need to instantiate the state stores
+   */
+  private ChainedOperators(MessageStream<M> source, TaskContext context) {
+    // create the pipeline/topology starting from source
+    // pass in the context s.t. stateful stream operators can initialize their stores
+  }
+
+  /**
+   * Static method to create a {@link ChainedOperators} from the {@code source} stream
+   *
+   * @param source  the input source {@link MessageStream}
+   * @param context  the {@link TaskContext} object used to initialize the {@link StateStoreImpl}
+   * @param <M>  the type of input {@link Message}
+   * @return a {@link ChainedOperators} object takes the {@code source} as input
+   */
+  public static <M extends Message> ChainedOperators create(MessageStream<M> source, TaskContext context) {
+    return new ChainedOperators<>(source, context);
+  }
+
+  /**
+   * Method to navigate the incoming {@code message} through the processing chains
+   *
+   * @param message  the incoming message to this {@link ChainedOperators}
+   * @param collector  the {@link MessageCollector} object within the process context
+   * @param coordinator  the {@link TaskCoordinator} object within the process context
+   */
+  public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
+    // TODO: add implementation of onNext() that actually triggers the process pipeline
+  }
+
+  public void onTimer(MessageCollector collector, TaskCoordinator coordinator) {
+    // TODO: add implementation of onTimer() that actually calls the corresponding window operator's onTimer() methods
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/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
new file mode 100644
index 0000000..81a7ede
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.api.data.Message;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+import org.reactivestreams.Processor;
+import org.reactivestreams.Subscriber;
+import org.reactivestreams.Subscription;
+
+import java.util.HashSet;
+import java.util.Set;
+
+
+/**
+ * Abstract base class for all stream operator implementation classes.
+ */
+public abstract class OperatorImpl<M extends Message, RM extends Message>
+    implements Processor<ProcessorContext<M>, ProcessorContext<RM>> {
+
+  private final Set<Subscriber<? super ProcessorContext<RM>>> subscribers = new HashSet<>();
+
+  @Override public void subscribe(Subscriber<? super ProcessorContext<RM>> s) {
+    // Only add once
+    subscribers.add(s);
+  }
+
+  @Override public void onSubscribe(Subscription s) {
+
+  }
+
+  @Override public void onNext(ProcessorContext<M> o) {
+
+    onNext(o.getMessage(), o.getCollector(), o.getCoordinator());
+  }
+
+  @Override public void onError(Throwable t) {
+
+  }
+
+  @Override public void onComplete() {
+
+  }
+
+  /**
+   * Each sub-class will implement this method to actually perform the transformation and call the downstream subscribers.
+   *
+   * @param message  the input {@link Message}
+   * @param collector  the {@link MessageCollector} in the context
+   * @param coordinator  the {@link TaskCoordinator} in the context
+   */
+  protected abstract void onNext(M message, MessageCollector collector, TaskCoordinator coordinator);
+
+  /**
+   * Stateful operators will need to override this method to initialize the operators
+   *
+   * @param context  the task context to initialize the operators within
+   */
+  protected void init(TaskContext context) {};
+
+  /**
+   * Method to trigger all downstream operators that consumes the output {@link org.apache.samza.operators.api.MessageStream}
+   * from this operator
+   *
+   * @param omsg  output {@link Message}
+   * @param collector  the {@link MessageCollector} in the context
+   * @param coordinator  the {@link TaskCoordinator} in the context
+   */
+  protected void nextProcessors(RM omsg, MessageCollector collector, TaskCoordinator coordinator) {
+    subscribers.forEach(sub ->
+      sub.onNext(new ProcessorContext<>(omsg, collector, coordinator))
+    );
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
new file mode 100644
index 0000000..b29d9c8
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.impl;
+
+import org.apache.samza.operators.api.data.Message;
+import org.apache.samza.operators.api.internal.Operators.StreamOperator;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+
+import java.util.Collection;
+import java.util.function.Function;
+
+
+/**
+ * Base class for all implementation of operators
+ *
+ * @param <M>  type of message in the input stream
+ * @param <RM>  type of message in the output stream
+ */
+public class SimpleOperatorImpl<M extends Message, RM extends Message> extends OperatorImpl<M, RM> {
+
+  private final Function<M, Collection<RM>> transformFn;
+
+  SimpleOperatorImpl(StreamOperator<M, RM> op) {
+    super();
+    this.transformFn = op.getFunction();
+  }
+
+  @Override protected void onNext(M imsg, MessageCollector collector, TaskCoordinator coordinator) {
+    // actually calling the transform function and then for each output, call nextProcessors()
+    this.transformFn.apply(imsg).forEach(r -> this.nextProcessors(r, collector, coordinator));
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/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
new file mode 100644
index 0000000..5d25cfa
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.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.impl;
+
+import org.apache.samza.operators.api.internal.Operators.SinkOperator;
+import org.apache.samza.operators.api.MessageStream;
+import org.apache.samza.operators.api.data.Message;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+
+
+/**
+ * Implementation for {@link SinkOperator}
+ */
+public class SinkOperatorImpl<M extends Message> extends OperatorImpl<M, Message> {
+  private final MessageStream.VoidFunction3<M, MessageCollector, TaskCoordinator> sinkFunc;
+
+  SinkOperatorImpl(SinkOperator<M> sinkOp) {
+    this.sinkFunc = sinkOp.getFunction();
+  }
+
+  @Override protected void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
+    this.sinkFunc.apply(message, collector, coordinator);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
new file mode 100644
index 0000000..f573fd0
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.impl;
+
+import org.apache.samza.operators.api.data.Message;
+import org.apache.samza.operators.api.internal.Operators.StoreFunctions;
+import org.apache.samza.storage.kv.Entry;
+import org.apache.samza.storage.kv.KeyValueStore;
+import org.apache.samza.task.TaskContext;
+
+
+/**
+ * The base class for all state stores
+ */
+public class StateStoreImpl<M extends Message, SK, SS> {
+  private final String storeName;
+  private final StoreFunctions<M, SK, SS> storeFunctions;
+  private KeyValueStore<SK, SS> kvStore = null;
+
+  public StateStoreImpl(StoreFunctions<M, SK, SS> store, String storeName) {
+    this.storeFunctions = store;
+    this.storeName = storeName;
+  }
+
+  public void init(TaskContext context) {
+    this.kvStore = (KeyValueStore<SK, SS>) context.getStore(this.storeName);
+  }
+
+  public Entry<SK, SS> getState(M m) {
+    SK key = this.storeFunctions.getStoreKeyFinder().apply(m);
+    SS state = this.kvStore.get(key);
+    return new Entry<>(key, state);
+  }
+
+  public Entry<SK, SS> updateState(M m, Entry<SK, SS> oldEntry) {
+    SS newValue = this.storeFunctions.getStateUpdater().apply(m, oldEntry.getValue());
+    this.kvStore.put(oldEntry.getKey(), newValue);
+    return new Entry<>(oldEntry.getKey(), newValue);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/impl/data/avro/AvroData.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/data/avro/AvroData.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/data/avro/AvroData.java
new file mode 100644
index 0000000..e4f5d79
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/data/avro/AvroData.java
@@ -0,0 +1,262 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.data.avro;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.avro.generic.GenericArray;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.samza.operators.api.data.Data;
+import org.apache.samza.operators.api.data.Schema;
+
+
+public class AvroData implements Data {
+  protected final Object datum;
+  protected final AvroSchema schema;
+
+  private AvroData(AvroSchema schema, Object datum) {
+    this.datum = datum;
+    this.schema = schema;
+  }
+
+  @Override
+  public Schema schema() {
+    return this.schema;
+  }
+
+  @Override
+  public Object value() {
+    return this.datum;
+  }
+
+  @Override
+  public int intValue() {
+    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
+  }
+
+  @Override
+  public long longValue() {
+    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
+  }
+
+  @Override
+  public float floatValue() {
+    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
+  }
+
+  @Override
+  public double doubleValue() {
+    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
+  }
+
+  @Override
+  public boolean booleanValue() {
+    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
+  }
+
+  @Override
+  public String strValue() {
+    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
+  }
+
+  @Override
+  public byte[] bytesValue() {
+    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
+  }
+
+  @Override
+  public List<Object> arrayValue() {
+    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
+  }
+
+  @Override
+  public Map<Object, Object> mapValue() {
+    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
+  }
+
+  @Override
+  public Data getElement(int index) {
+    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
+  }
+
+  @Override
+  public Data getFieldData(String fldName) {
+    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
+  }
+
+  public static AvroData getArray(AvroSchema schema, Object datum) {
+    if (schema.getType() != Schema.Type.ARRAY) {
+      throw new IllegalArgumentException("Can't create an array object with non-array schema:" + schema.getType());
+    }
+    return new AvroData(schema, datum) {
+      @SuppressWarnings("unchecked")
+      private final GenericArray<Object> array = (GenericArray<Object>) this.datum;
+
+      @Override
+      public List<Object> arrayValue() {
+        return this.array;
+      }
+
+      @Override
+      public Data getElement(int index) {
+        return this.schema.getElementType().read(array.get(index));
+      }
+
+    };
+  }
+
+  public static AvroData getMap(AvroSchema schema, Object datum) {
+    if (schema.getType() != Schema.Type.MAP) {
+      throw new IllegalArgumentException("Can't create a map object with non-map schema:" + schema.getType());
+    }
+    return new AvroData(schema, datum) {
+      @SuppressWarnings("unchecked")
+      private final Map<Object, Object> map = (Map<Object, Object>) datum;
+
+      @Override
+      public Map<Object, Object> mapValue() {
+        return this.map;
+      }
+
+      @Override
+      public Data getFieldData(String fldName) {
+        // TODO Auto-generated method stub
+        return this.schema.getValueType().read(map.get(fldName));
+      }
+
+    };
+  }
+
+  public static AvroData getStruct(AvroSchema schema, Object datum) {
+    if (schema.getType() != Schema.Type.STRUCT) {
+      throw new IllegalArgumentException("Can't create a struct object with non-struct schema:" + schema.getType());
+    }
+    return new AvroData(schema, datum) {
+      private final GenericRecord record = (GenericRecord) datum;
+
+      @Override
+      public Data getFieldData(String fldName) {
+        // TODO Auto-generated method stub
+        return this.schema.getFieldType(fldName).read(record.get(fldName));
+      }
+
+    };
+  }
+
+  public static AvroData getInt(AvroSchema schema, Object datum) {
+    if (schema.getType() != Schema.Type.INTEGER || !(datum instanceof Integer)) {
+      throw new IllegalArgumentException("data object and schema mismatch. schema:" + schema.getType() + ", data: "
+          + datum.getClass().getName());
+    }
+    return new AvroData(schema, datum) {
+      @Override
+      public int intValue() {
+        return ((Integer) datum).intValue();
+      }
+
+    };
+  }
+
+  public static AvroData getLong(AvroSchema schema, Object datum) {
+    if (schema.getType() != Schema.Type.LONG || !(datum instanceof Long)) {
+      throw new IllegalArgumentException("data object and schema mismatch. schema:" + schema.getType() + ", data: "
+          + datum.getClass().getName());
+    }
+    return new AvroData(schema, datum) {
+      @Override
+      public long longValue() {
+        return ((Long) datum).longValue();
+      }
+
+    };
+  }
+
+  public static AvroData getFloat(AvroSchema schema, Object datum) {
+    if (schema.getType() != Schema.Type.FLOAT || !(datum instanceof Float)) {
+      throw new IllegalArgumentException("data object and schema mismatch. schema:" + schema.getType() + ", data: "
+          + datum.getClass().getName());
+    }
+    return new AvroData(schema, datum) {
+      @Override
+      public float floatValue() {
+        return ((Float) datum).floatValue();
+      }
+
+    };
+  }
+
+  public static AvroData getDouble(AvroSchema schema, Object datum) {
+    if (schema.getType() != Schema.Type.DOUBLE || !(datum instanceof Double)) {
+      throw new IllegalArgumentException("data object and schema mismatch. schema:" + schema.getType() + ", data: "
+          + datum.getClass().getName());
+    }
+    return new AvroData(schema, datum) {
+      @Override
+      public double doubleValue() {
+        return ((Double) datum).doubleValue();
+      }
+
+    };
+  }
+
+  public static AvroData getBoolean(AvroSchema schema, Object datum) {
+    if (schema.getType() != Schema.Type.BOOLEAN || !(datum instanceof Boolean)) {
+      throw new IllegalArgumentException("data object and schema mismatch. schema:" + schema.getType() + ", data: "
+          + datum.getClass().getName());
+    }
+    return new AvroData(schema, datum) {
+      @Override
+      public boolean booleanValue() {
+        return ((Boolean) datum).booleanValue();
+      }
+
+    };
+  }
+
+  public static AvroData getString(AvroSchema schema, Object datum) {
+    if (schema.getType() != Schema.Type.STRING || !(datum instanceof CharSequence)) {
+      throw new IllegalArgumentException("data object and schema mismatch. schema:" + schema.getType() + ", data: "
+          + datum.getClass().getName());
+    }
+    return new AvroData(schema, datum) {
+      @Override
+      public String strValue() {
+        return ((CharSequence) datum).toString();
+      }
+
+    };
+  }
+
+  public static AvroData getBytes(AvroSchema schema, Object datum) {
+    if (schema.getType() != Schema.Type.BYTES || !(datum instanceof ByteBuffer)) {
+      throw new IllegalArgumentException("data object and schema mismatch. schema:" + schema.getType() + ", data: "
+          + datum.getClass().getName());
+    }
+    return new AvroData(schema, datum) {
+      @Override
+      public byte[] bytesValue() {
+        return ((ByteBuffer) datum).array();
+      }
+
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/impl/data/avro/AvroSchema.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/data/avro/AvroSchema.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/data/avro/AvroSchema.java
new file mode 100644
index 0000000..c04e4f6
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/data/avro/AvroSchema.java
@@ -0,0 +1,296 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.data.avro;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.avro.Schema.Field;
+import org.apache.samza.operators.api.data.Data;
+import org.apache.samza.operators.api.data.Schema;
+
+
+public class AvroSchema implements Schema {
+
+  protected final org.apache.avro.Schema avroSchema;
+  protected final Schema.Type type;
+
+  private final static Map<org.apache.avro.Schema.Type, AvroSchema> primSchemas =
+      new HashMap<org.apache.avro.Schema.Type, AvroSchema>();
+
+  static {
+    primSchemas.put(org.apache.avro.Schema.Type.INT,
+        new AvroSchema(org.apache.avro.Schema.create(org.apache.avro.Schema.Type.INT)) {
+      @Override
+      public Data read(Object datum) {
+        return AvroData.getInt(this, datum);
+      }
+    });
+    primSchemas.put(org.apache.avro.Schema.Type.LONG,
+        new AvroSchema(org.apache.avro.Schema.create(org.apache.avro.Schema.Type.LONG)) {
+      @Override
+      public Data read(Object datum) {
+        return AvroData.getLong(this, datum);
+      }
+    });
+    primSchemas.put(org.apache.avro.Schema.Type.FLOAT,
+        new AvroSchema(org.apache.avro.Schema.create(org.apache.avro.Schema.Type.FLOAT)) {
+      @Override
+      public Data read(Object datum) {
+        return AvroData.getFloat(this, datum);
+      }
+    });
+    primSchemas.put(org.apache.avro.Schema.Type.DOUBLE,
+        new AvroSchema(org.apache.avro.Schema.create(org.apache.avro.Schema.Type.DOUBLE)) {
+      @Override
+      public Data read(Object datum) {
+        return AvroData.getDouble(this, datum);
+      }
+    });
+    primSchemas.put(org.apache.avro.Schema.Type.BOOLEAN,
+        new AvroSchema(org.apache.avro.Schema.create(org.apache.avro.Schema.Type.BOOLEAN)) {
+      @Override
+      public Data read(Object datum) {
+        return AvroData.getBoolean(this, datum);
+      }
+    });
+    primSchemas.put(org.apache.avro.Schema.Type.STRING,
+        new AvroSchema(org.apache.avro.Schema.create(org.apache.avro.Schema.Type.STRING)) {
+      @Override
+      public Data read(Object datum) {
+        return AvroData.getString(this, datum);
+      }
+    });
+    primSchemas.put(org.apache.avro.Schema.Type.BYTES,
+        new AvroSchema(org.apache.avro.Schema.create(org.apache.avro.Schema.Type.BYTES)) {
+      @Override
+      public Data read(Object datum) {
+        return AvroData.getBytes(this, datum);
+      }
+    });
+  };
+
+  public static AvroSchema getSchema(final org.apache.avro.Schema schema) {
+    Schema.Type type = mapType(schema.getType());
+    if (type != Schema.Type.ARRAY && type != Schema.Type.MAP && type != Schema.Type.STRUCT) {
+      return primSchemas.get(schema.getType());
+    }
+    // otherwise, construct the new schema
+    // TODO: It would be possible to assign each complex schema an ID and cache it w/o repeated create in-memory schema objects
+    switch (type) {
+      case ARRAY:
+        return new AvroSchema(schema) {
+          @Override
+          public Data transform(Data input) {
+            // This would get all the elements until the length of the current schema's array length
+            if (input.schema().getType() != Schema.Type.ARRAY) {
+              throw new IllegalArgumentException("Schema mismatch. Can't transfer data. input schema: "
+                  + input.schema().getType());
+            }
+            if (!input.schema().getElementType().equals(this.getElementType())) {
+              throw new IllegalArgumentException("Element schema mismatch. Can't transfer data. input schema: "
+                  + input.schema().getElementType().getType());
+            }
+            // input type matches array type
+            return AvroData.getArray(this, input.value());
+          }
+        };
+      case MAP:
+        return new AvroSchema(schema) {
+          @Override
+          public Data transform(Data input) {
+            // This would get all the elements until the length of the current schema's array length
+            if (input.schema().getType() != Schema.Type.MAP) {
+              throw new IllegalArgumentException("Schema mismatch. Can't transfer data. input schema: "
+                  + input.schema().getType());
+            }
+            if (!input.schema().getValueType().equals(this.getValueType())) {
+              throw new IllegalArgumentException("Element schema mismatch. Can't transfer data. input schema: "
+                  + input.schema().getValueType().getType());
+            }
+            // input type matches map type
+            return AvroData.getMap(this, input.value());
+          }
+        };
+      case STRUCT:
+        return new AvroSchema(schema) {
+          @SuppressWarnings("serial")
+          private final Map<String, Schema> fldSchemas = new HashMap<String, Schema>() {
+            {
+              for (Field field : schema.getFields()) {
+                put(field.name(), getSchema(field.schema()));
+              }
+            }
+          };
+
+          @Override
+          public Map<String, Schema> getFields() {
+            return this.fldSchemas;
+          }
+
+          @Override
+          public Schema getFieldType(String fldName) {
+            return this.fldSchemas.get(fldName);
+          }
+
+          @Override
+          public Data transform(Data input) {
+            // This would get all the elements until the length of the current schema's array length
+            if (input.schema().getType() != Schema.Type.STRUCT) {
+              throw new IllegalArgumentException("Schema mismatch. Can't transfer data. input schema: "
+                  + input.schema().getType());
+            }
+            // Note: this particular transform function only implements "projection to a sub-set" concept.
+            // More complex function is needed if some other concepts such as "merge from two sets of data", "allow null if does not exist" are needed
+            for (String fldName : this.fldSchemas.keySet()) {
+              // check each field schema matches input
+              Schema fldSchema = this.fldSchemas.get(fldName);
+              Schema inputFld = input.schema().getFieldType(fldName);
+              if (!fldSchema.equals(inputFld)) {
+                throw new IllegalArgumentException("Field schema mismatch. Can't transfer data for field " + fldName
+                    + ". input field schema:" + inputFld.getType() + ", this field schema: " + fldSchema.getType());
+              }
+            }
+            // input type matches struct type
+            return AvroData.getStruct(this, input.value());
+          }
+
+        };
+      default:
+        throw new IllegalArgumentException("Un-recognized complext data type:" + type);
+    }
+  }
+
+  private AvroSchema(org.apache.avro.Schema schema) {
+    this.avroSchema = schema;
+    this.type = mapType(schema.getType());
+  }
+
+  private static Type mapType(org.apache.avro.Schema.Type type) {
+    switch (type) {
+      case ARRAY:
+        return Schema.Type.ARRAY;
+      case RECORD:
+        return Schema.Type.STRUCT;
+      case MAP:
+        return Schema.Type.MAP;
+      case INT:
+        return Schema.Type.INTEGER;
+      case LONG:
+        return Schema.Type.LONG;
+      case BOOLEAN:
+        return Schema.Type.BOOLEAN;
+      case FLOAT:
+        return Schema.Type.FLOAT;
+      case DOUBLE:
+        return Schema.Type.DOUBLE;
+      case STRING:
+        return Schema.Type.STRING;
+      case BYTES:
+        return Schema.Type.BYTES;
+      default:
+        throw new IllegalArgumentException("Avro schema: " + type + " is not supported");
+    }
+  }
+
+  @Override
+  public Type getType() {
+    return this.type;
+  }
+
+  @Override
+  public Schema getElementType() {
+    if (this.type != Schema.Type.ARRAY) {
+      throw new UnsupportedOperationException("Can't getElmentType with non-array schema: " + this.type);
+    }
+    return getSchema(this.avroSchema.getElementType());
+  }
+
+  @Override
+  public Schema getValueType() {
+    if (this.type != Schema.Type.MAP) {
+      throw new UnsupportedOperationException("Can't getValueType with non-map schema: " + this.type);
+    }
+    return getSchema(this.avroSchema.getValueType());
+  }
+
+  @Override
+  public Map<String, Schema> getFields() {
+    throw new UnsupportedOperationException("Can't get field types with unknown schema type:" + this.type);
+  }
+
+  @Override
+  public Schema getFieldType(String fldName) {
+    throw new UnsupportedOperationException("Can't getFieldType with non-map/non-struct schema: " + this.type);
+  }
+
+  @Override
+  public Data read(Object object) {
+    if (this.avroSchema.getType() == org.apache.avro.Schema.Type.ARRAY) {
+      return AvroData.getArray(this, object);
+    } else if (this.avroSchema.getType() == org.apache.avro.Schema.Type.MAP) {
+      return AvroData.getMap(this, object);
+    } else if (this.avroSchema.getType() == org.apache.avro.Schema.Type.RECORD) {
+      return AvroData.getStruct(this, object);
+    }
+    throw new UnsupportedOperationException("Reading unknown complext type:" + this.type + " is not supported");
+  }
+
+  @Override
+  public Data transform(Data inputData) {
+    if (inputData.schema().getType() == Schema.Type.ARRAY || inputData.schema().getType() == Schema.Type.MAP
+        || inputData.schema().getType() == Schema.Type.STRUCT) {
+      throw new IllegalArgumentException("Complex schema should have overriden the default transform() function.");
+    }
+    if (inputData.schema().getType() != this.type) {
+      throw new IllegalArgumentException("Can't transform a mismatched primitive type. this type:" + this.type
+          + ", input type:" + inputData.schema().getType());
+    }
+    return inputData;
+  }
+
+  @Override
+  public boolean equals(Schema other) {
+    // TODO Auto-generated method stub
+    if (this.type != other.getType()) {
+      return false;
+    }
+    switch (this.type) {
+      case ARRAY:
+        // check if element types are the same
+        return this.getElementType().equals(other.getElementType());
+      case MAP:
+        // check if value types are the same
+        return this.getValueType().equals(other.getValueType());
+      case STRUCT:
+        // check if the fields schemas in this equals the other
+        // NOTE: this equals check is in consistent with the "projection to subset" concept implemented in transform()
+        for (String fieldName : this.getFields().keySet()) {
+          if (!this.getFieldType(fieldName).equals(other.getFieldType(fieldName))) {
+            return false;
+          }
+        }
+        return true;
+      default:
+        return true;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/impl/data/serializers/SqlAvroSerde.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/data/serializers/SqlAvroSerde.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/data/serializers/SqlAvroSerde.java
new file mode 100644
index 0000000..2432aca
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/data/serializers/SqlAvroSerde.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.operators.impl.data.serializers;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.io.Encoder;
+import org.apache.avro.io.BinaryEncoder;
+import org.apache.samza.SamzaException;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.operators.impl.data.avro.AvroData;
+import org.apache.samza.operators.impl.data.avro.AvroSchema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+public class SqlAvroSerde implements Serde<AvroData> {
+  private static Logger log = LoggerFactory.getLogger(SqlAvroSerde.class);
+
+  private final Schema avroSchema;
+  private final GenericDatumReader<GenericRecord> reader;
+  private final GenericDatumWriter<Object> writer;
+
+  public SqlAvroSerde(Schema avroSchema) {
+    this.avroSchema = avroSchema;
+    this.reader = new GenericDatumReader<GenericRecord>(avroSchema);
+    this.writer = new GenericDatumWriter<Object>(avroSchema);
+  }
+
+  @Override
+  public AvroData fromBytes(byte[] bytes) {
+    GenericRecord data;
+
+    try {
+      data = reader.read(null, DecoderFactory.defaultFactory().createBinaryDecoder(bytes, null));
+      return getAvroData(data, avroSchema);
+    } catch (IOException e) {
+      String errMsg = "Cannot decode message.";
+      log.error(errMsg, e);
+      throw new SamzaException(errMsg, e);
+    }
+  }
+
+  @Override
+  public byte[] toBytes(AvroData object) {
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+    Encoder encoder = new BinaryEncoder(out);
+
+    try {
+      writer.write(object.value(), encoder);
+      encoder.flush();
+      return out.toByteArray();
+    } catch (IOException e) {
+      String errMsg = "Cannot perform Avro binary encode.";
+      log.error(errMsg, e);
+      throw new SamzaException(errMsg, e);
+    }
+  }
+
+  private AvroData getAvroData(GenericRecord data, Schema type){
+    AvroSchema schema = AvroSchema.getSchema(type);
+    switch (type.getType()){
+      case RECORD:
+        return AvroData.getStruct(schema, data);
+      case ARRAY:
+        return AvroData.getArray(schema, data);
+      case MAP:
+        return AvroData.getMap(schema, data);
+      case INT:
+        return AvroData.getInt(schema, data);
+      case LONG:
+        return AvroData.getLong(schema, data);
+      case BOOLEAN:
+        return AvroData.getBoolean(schema, data);
+      case FLOAT:
+        return AvroData.getFloat(schema, data);
+      case DOUBLE:
+        return AvroData.getDouble(schema, data);
+      case STRING:
+        return AvroData.getString(schema, data);
+      case BYTES:
+        return AvroData.getBytes(schema, data);
+      default:
+        throw new IllegalArgumentException("Avro schema: " + type + " is not supported");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/impl/data/serializers/SqlAvroSerdeFactory.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/data/serializers/SqlAvroSerdeFactory.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/data/serializers/SqlAvroSerdeFactory.java
new file mode 100644
index 0000000..edd8859
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/data/serializers/SqlAvroSerdeFactory.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.impl.data.serializers;
+
+import org.apache.avro.Schema;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.Config;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.serializers.SerdeFactory;
+import org.apache.samza.operators.impl.data.avro.AvroData;
+
+public class SqlAvroSerdeFactory implements SerdeFactory<AvroData> {
+  public static final String PROP_AVRO_SCHEMA = "serializers.%s.schema";
+
+  @Override
+  public Serde<AvroData> getSerde(String name, Config config) {
+    String avroSchemaStr = config.get(String.format(PROP_AVRO_SCHEMA, name));
+    if (avroSchemaStr == null || avroSchemaStr.isEmpty()) {
+      throw new SamzaException("Cannot find avro schema for SerdeFactory '" + name + "'.");
+    }
+
+    return new SqlAvroSerde(Schema.parse(avroSchemaStr));
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/impl/data/serializers/SqlStringSerde.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/data/serializers/SqlStringSerde.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/data/serializers/SqlStringSerde.java
new file mode 100644
index 0000000..1267ab6
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/data/serializers/SqlStringSerde.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.impl.data.serializers;
+
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.operators.impl.data.string.StringData;
+
+
+public class SqlStringSerde implements Serde<StringData> {
+
+    private final Serde<String> serde;
+
+    public SqlStringSerde(String encoding) {
+        this.serde = new StringSerde(encoding);
+    }
+
+    @Override
+    public StringData fromBytes(byte[] bytes) {
+          return new StringData(serde.fromBytes(bytes));
+    }
+
+    @Override
+    public byte[] toBytes(StringData object) {
+        return serde.toBytes(object.strValue());
+    }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/impl/data/serializers/SqlStringSerdeFactory.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/data/serializers/SqlStringSerdeFactory.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/data/serializers/SqlStringSerdeFactory.java
new file mode 100644
index 0000000..3b6a3e0
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/data/serializers/SqlStringSerdeFactory.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.impl.data.serializers;
+
+
+import org.apache.samza.config.Config;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.serializers.SerdeFactory;
+import org.apache.samza.operators.impl.data.string.StringData;
+
+public class SqlStringSerdeFactory implements SerdeFactory<StringData> {
+    @Override
+    public Serde<StringData> getSerde(String name, Config config) {
+        return new SqlStringSerde(config.get("encoding", "UTF-8"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/impl/data/string/StringData.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/data/string/StringData.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/data/string/StringData.java
new file mode 100644
index 0000000..86e9917
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/data/string/StringData.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.operators.impl.data.string;
+
+import org.apache.samza.operators.api.data.Data;
+import org.apache.samza.operators.api.data.Schema;
+
+import java.util.List;
+import java.util.Map;
+
+public class StringData implements Data {
+    private final Object datum;
+    private final Schema schema;
+
+    public StringData(Object datum) {
+        this.datum = datum;
+        this.schema = new StringSchema();
+    }
+
+    @Override
+    public Schema schema() {
+        return this.schema;
+    }
+
+    @Override
+    public Object value() {
+        return this.datum;
+    }
+
+    @Override
+    public int intValue() {
+        throw new UnsupportedOperationException("Can't get int value for a string type data");
+    }
+
+    @Override
+    public long longValue() {
+        throw new UnsupportedOperationException("Can't get long value for a string type data");
+    }
+
+    @Override
+    public float floatValue() {
+        throw new UnsupportedOperationException("Can't get float value for a string type data");
+    }
+
+    @Override
+    public double doubleValue() {
+        throw new UnsupportedOperationException("Can't get double value for a string type data");
+    }
+
+    @Override
+    public boolean booleanValue() {
+        throw new UnsupportedOperationException("Can't get boolean value for a string type data");
+    }
+
+    @Override
+    public String strValue() {
+        return String.valueOf(datum);
+    }
+
+    @Override
+    public byte[] bytesValue() {
+        throw new UnsupportedOperationException("Can't get bytesValue for a string type data");
+    }
+
+    @Override
+    public List<Object> arrayValue() {
+        throw new UnsupportedOperationException("Can't get arrayValue for a string type data");
+    }
+
+    @Override
+    public Map<Object, Object> mapValue() {
+        throw new UnsupportedOperationException("Can't get mapValue for a string type data");
+    }
+
+    @Override
+    public Data getElement(int index) {
+        throw new UnsupportedOperationException("Can't getElement(index) on a string type data");
+    }
+
+    @Override
+    public Data getFieldData(String fldName) {
+        throw new UnsupportedOperationException("Can't getFieldData(fieldName) for a string type data");
+    }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/impl/data/string/StringSchema.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/data/string/StringSchema.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/data/string/StringSchema.java
new file mode 100644
index 0000000..b19dfeb
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/data/string/StringSchema.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.operators.impl.data.string;
+
+import org.apache.samza.operators.api.data.Data;
+import org.apache.samza.operators.api.data.Schema;
+
+import java.util.Map;
+
+public class StringSchema implements Schema {
+    private Type type = Type.STRING;
+
+    @Override
+    public Type getType() {
+      return Type.STRING;
+    }
+
+    @Override
+    public Schema getElementType() {
+      throw new UnsupportedOperationException("Can't getElmentType with non-array schema: " + this.type);
+    }
+
+    @Override
+    public Schema getValueType() {
+        throw new UnsupportedOperationException("Can't getValueType with non-map schema: " + this.type);
+    }
+
+    @Override
+    public Map<String, Schema> getFields() {
+        throw new UnsupportedOperationException("Can't get field types with unknown schema type:" + this.type);
+    }
+
+    @Override
+    public Schema getFieldType(String fldName) {
+        throw new UnsupportedOperationException("Can't getFieldType with non-map/non-struct schema: " + this.type);
+    }
+
+    @Override
+    public Data read(Object object) {
+        return new StringData(object);
+    }
+
+    @Override
+    public Data transform(Data inputData) {
+        if (inputData.schema().getType() != this.type) {
+            throw new IllegalArgumentException("Can't transform a mismatched primitive type. this type:" + this.type
+                    + ", input type:" + inputData.schema().getType());
+        }
+        return inputData;
+    }
+
+    @Override
+    public boolean equals(Schema other) {
+        return other.getType() == this.type;
+    }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
new file mode 100644
index 0000000..2de53aa
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.impl.window;
+
+import org.apache.samza.operators.api.MessageStream;
+import org.apache.samza.operators.api.WindowState;
+import org.apache.samza.operators.api.data.Message;
+import org.apache.samza.operators.api.internal.Operators.WindowOperator;
+import org.apache.samza.operators.api.internal.WindowOutput;
+import org.apache.samza.operators.impl.OperatorImpl;
+import org.apache.samza.operators.impl.StateStoreImpl;
+import org.apache.samza.storage.kv.Entry;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+
+import java.util.function.BiFunction;
+
+
+/**
+ * Default implementation class of a {@link WindowOperator} for a session window.
+ *
+ * @param <M>  the type of input {@link Message}
+ * @param <RK>  the type of window key
+ * @param <RM>  the type of aggregated value of the window
+ */
+public class SessionWindowImpl<M extends Message, RK, WS extends WindowState, RM extends WindowOutput<RK, ?>> extends
+    OperatorImpl<M, RM> {
+  private final BiFunction<M, Entry<RK, WS>, RM> txfmFunction;
+  private final StateStoreImpl<M, RK, WS> wndStore;
+
+  SessionWindowImpl(WindowOperator<M, RK, WS, RM> sessWnd, MessageStream<M> input) {
+    this.txfmFunction = sessWnd.getFunction();
+    this.wndStore = new StateStoreImpl<>(sessWnd.getStoreFunctions(), sessWnd.getStoreName(input));
+  }
+
+  @Override protected void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
+    Entry<RK, WS> state = this.wndStore.getState(message);
+    this.nextProcessors(this.txfmFunction.apply(message, state), collector, coordinator);
+    this.wndStore.updateState(message, state);
+  }
+
+  public void onTimer(MessageCollector collector, TaskCoordinator coordinator) {
+    // This is to periodically check the timeout triggers to get the list of window states to be updated
+  }
+
+  @Override protected void init(TaskContext context) {
+    this.wndStore.init(context);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java b/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
new file mode 100644
index 0000000..e340fe8
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.task;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.api.MessageStream;
+import org.apache.samza.operators.api.MessageStreams;
+import org.apache.samza.operators.api.MessageStreams.SystemMessageStream;
+import org.apache.samza.operators.api.data.IncomingSystemMessage;
+import org.apache.samza.operators.impl.ChainedOperators;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.util.HashMap;
+import java.util.Map;
+
+
+/**
+ * An adaptor task class that invoke the user-implemented (@link StreamOperatorTask}s via {@link MessageStream} programming APIs
+ *
+ */
+public final class StreamOperatorAdaptorTask implements StreamTask, InitableTask, WindowableTask {
+  /**
+   * A map with entries mapping {@link SystemStreamPartition} to {@link org.apache.samza.operators.impl.ChainedOperators} that takes the {@link SystemStreamPartition}
+   * as the input stream
+   */
+  private final Map<SystemStreamPartition, ChainedOperators> operatorChains = new HashMap<>();
+
+  /**
+   * Wrapped {@link StreamOperatorTask} class
+   */
+  private final StreamOperatorTask  userTask;
+
+  /**
+   * Constructor that wraps the user-defined {@link StreamOperatorTask}
+   *
+   * @param userTask  the user-defined {@link StreamOperatorTask}
+   */
+  public StreamOperatorAdaptorTask(StreamOperatorTask userTask) {
+    this.userTask = userTask;
+  }
+
+  @Override
+  public final void init(Config config, TaskContext context) throws Exception {
+    if (this.userTask instanceof InitableTask) {
+      ((InitableTask) this.userTask).init(config, context);
+    }
+    Map<SystemStreamPartition, SystemMessageStream> sources = new HashMap<>();
+    context.getSystemStreamPartitions().forEach(ssp -> {
+      SystemMessageStream ds = MessageStreams.input(ssp);
+      sources.put(ssp, ds);
+    });
+    this.userTask.initOperators(sources.values());
+    sources.forEach((ssp, ds) -> operatorChains.put(ssp, ChainedOperators.create(ds, context)));
+  }
+
+  @Override
+  public final void process(IncomingMessageEnvelope ime, MessageCollector collector, TaskCoordinator coordinator) {
+    this.operatorChains.get(ime.getSystemStreamPartition()).onNext(new IncomingSystemMessage(ime), collector, coordinator);
+  }
+
+  @Override
+  public final void window(MessageCollector collector, TaskCoordinator coordinator) throws Exception{
+    this.operatorChains.forEach((ssp, chain) -> chain.onTimer(collector, coordinator));
+    if (this.userTask instanceof WindowableTask) {
+      ((WindowableTask) this.userTask).window(collector, coordinator);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/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..cfdb694
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.task;
+
+import org.apache.samza.operators.api.MessageStreams.SystemMessageStream;
+import java.util.Collection;
+
+/**
+ * This interface defines the methods that user needs to implement via the operator programming APIs.
+ */
+public interface StreamOperatorTask {
+
+  /**
+   * Defines the method for users to initialize the operator chains consuming from all {@link SystemMessageStream}s.
+   * Users have to implement this function to instantiate {@link org.apache.samza.operators.impl.ChainedOperators} that
+   * will process each incoming {@link SystemMessageStream}.
+   *
+   * Note that each {@link SystemMessageStream} corresponds to an input {@link org.apache.samza.system.SystemStreamPartition}
+   *
+   * @param sources  the collection of {@link SystemMessageStream}s that takes {@link org.apache.samza.operators.api.data.IncomingSystemMessage}
+   *                 from a {@link org.apache.samza.system.SystemStreamPartition}
+   */
+  void initOperators(Collection<SystemMessageStream> sources);
+
+
+}

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

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/test/java/org/apache/samza/operators/api/TestMessageStreams.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/api/TestMessageStreams.java b/samza-operator/src/test/java/org/apache/samza/operators/api/TestMessageStreams.java
new file mode 100644
index 0000000..e6aa692
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/api/TestMessageStreams.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.api;
+
+import org.apache.samza.Partition;
+import org.apache.samza.system.SystemStreamPartition;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+
+public class TestMessageStreams {
+
+  @Test public void testInput() {
+    SystemStreamPartition ssp = new SystemStreamPartition("my-system", "my-stream", new Partition(0));
+    MessageStreams.SystemMessageStream mSysStream = MessageStreams.input(ssp);
+    assertEquals(mSysStream.getSystemStreamPartition(), ssp);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/test/java/org/apache/samza/operators/api/TestTriggerBuilder.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/api/TestTriggerBuilder.java b/samza-operator/src/test/java/org/apache/samza/operators/api/TestTriggerBuilder.java
new file mode 100644
index 0000000..8faa92c
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/api/TestTriggerBuilder.java
@@ -0,0 +1,211 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.api;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
+public class TestTriggerBuilder{
+  private Field earlyTriggerField;
+  private Field lateTriggerField;
+  private Field timerTriggerField;
+  private Field earlyTriggerUpdater;
+  private Field lateTriggerUpdater;
+
+  @Before
+  public void testPrep() throws Exception {
+    this.earlyTriggerField = TriggerBuilder.class.getDeclaredField("earlyTrigger");
+    this.lateTriggerField = TriggerBuilder.class.getDeclaredField("lateTrigger");
+    this.timerTriggerField = TriggerBuilder.class.getDeclaredField("timerTrigger");
+    this.earlyTriggerUpdater = TriggerBuilder.class.getDeclaredField("earlyTriggerUpdater");
+    this.lateTriggerUpdater = TriggerBuilder.class.getDeclaredField("lateTriggerUpdater");
+
+    this.earlyTriggerField.setAccessible(true);
+    this.lateTriggerField.setAccessible(true);
+    this.timerTriggerField.setAccessible(true);
+    this.earlyTriggerUpdater.setAccessible(true);
+    this.lateTriggerUpdater.setAccessible(true);
+  }
+
+  @Test public void testStaticCreators() throws NoSuchFieldException, IllegalAccessException {
+    TriggerBuilder<TestMessage, Collection<TestMessage>> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
+    BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean> triggerField =
+        (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
+    WindowState<Collection<TestMessage>> mockState = mock(WindowState.class);
+    when(mockState.getNumberMessages()).thenReturn(200L);
+    assertFalse(triggerField.apply(null, mockState));
+    when(mockState.getNumberMessages()).thenReturn(2000L);
+    assertTrue(triggerField.apply(null, mockState));
+
+    Function<TestMessage, Boolean> tokenFunc = m -> true;
+    builder = TriggerBuilder.earlyTriggerOnTokenMsg(tokenFunc);
+    triggerField = (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
+    TestMessage m = mock(TestMessage.class);
+    assertTrue(triggerField.apply(m, mockState));
+
+    builder = TriggerBuilder.earlyTriggerOnEventTime(TestMessage::getTimestamp, 30000L);
+    triggerField = (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
+    when(mockState.getEarliestEventTimeNs()).thenReturn(1000000000L);
+    when(mockState.getLatestEventTimeNs()).thenReturn(20000000000L);
+    when(m.getTimestamp()).thenReturn(19999000000L);
+    assertFalse(triggerField.apply(m, mockState));
+    when(m.getTimestamp()).thenReturn(32000000000L);
+    assertTrue(triggerField.apply(m, mockState));
+    when(m.getTimestamp()).thenReturn(1001000000L);
+    when(mockState.getLatestEventTimeNs()).thenReturn(32000000000L);
+    assertTrue(triggerField.apply(m, mockState));
+
+    BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean> mockFunc = mock(BiFunction.class);
+    builder = TriggerBuilder.earlyTrigger(mockFunc);
+    triggerField = (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
+    assertEquals(triggerField, mockFunc);
+
+    builder = TriggerBuilder.timeoutSinceFirstMessage(10000L);
+    Function<WindowState<Collection<TestMessage>>, Boolean> timerTrigger =
+        (Function<WindowState<Collection<TestMessage>>, Boolean>) this.timerTriggerField.get(builder);
+    when(mockState.getFirstMessageTimeNs()).thenReturn(0L);
+    assertTrue(timerTrigger.apply(mockState));
+    // set the firstMessageTimeNs to 9 second earlier, giving the test 1 second to fire up the timerTrigger before assertion
+    when(mockState.getFirstMessageTimeNs()).thenReturn(TimeUnit.MILLISECONDS.toNanos(System.currentTimeMillis() - 9000L));
+    assertFalse(timerTrigger.apply(mockState));
+
+    builder = TriggerBuilder.timeoutSinceLastMessage(10000L);
+    timerTrigger = (Function<WindowState<Collection<TestMessage>>, Boolean>) this.timerTriggerField.get(builder);
+    when(mockState.getLastMessageTimeNs()).thenReturn(0L);
+    assertTrue(timerTrigger.apply(mockState));
+    // set the lastMessageTimeNs to 9 second earlier, giving the test 1 second to fire up the timerTrigger before assertion
+    when(mockState.getLastMessageTimeNs()).thenReturn(TimeUnit.MILLISECONDS.toNanos(System.currentTimeMillis() - 9000));
+    assertFalse(timerTrigger.apply(mockState));
+  }
+
+  @Test public void testAddTimerTriggers() throws IllegalAccessException {
+    TriggerBuilder<TestMessage, Collection<TestMessage>> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
+    builder.addTimeoutSinceFirstMessage(10000L);
+    // exam that both earlyTrigger and timer triggers are set up
+    BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean> triggerField =
+        (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
+    WindowState<Collection<TestMessage>> mockState = mock(WindowState.class);
+    when(mockState.getNumberMessages()).thenReturn(200L);
+    assertFalse(triggerField.apply(null, mockState));
+    // check the timer trigger
+    Function<WindowState<Collection<TestMessage>>, Boolean> timerTrigger =
+        (Function<WindowState<Collection<TestMessage>>, Boolean>) this.timerTriggerField.get(builder);
+    when(mockState.getFirstMessageTimeNs()).thenReturn(0L);
+    assertTrue(timerTrigger.apply(mockState));
+    // set the firstMessageTimeNs to 9 second earlier, giving the test 1 second to fire up the timerTrigger before assertion
+    when(mockState.getFirstMessageTimeNs()).thenReturn(TimeUnit.MILLISECONDS.toNanos(System.currentTimeMillis() - 9000L));
+    assertFalse(timerTrigger.apply(mockState));
+
+    // exam that both early trigger and timer triggers are set up
+    builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
+    triggerField = (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
+    mockState = mock(WindowState.class);
+    when(mockState.getNumberMessages()).thenReturn(200L);
+    assertFalse(triggerField.apply(null, mockState));
+    builder.addTimeoutSinceLastMessage(20000L);
+    // check the timer trigger
+    timerTrigger = (Function<WindowState<Collection<TestMessage>>, Boolean>) this.timerTriggerField.get(builder);
+    when(mockState.getLastMessageTimeNs()).thenReturn(0L);
+    assertTrue(timerTrigger.apply(mockState));
+    // set the firstMessageTimeNs to 9 second earlier, giving the test 1 second to fire up the timerTrigger before assertion
+    when(mockState.getLastMessageTimeNs()).thenReturn(TimeUnit.MILLISECONDS.toNanos(System.currentTimeMillis() - 9000L));
+    assertFalse(timerTrigger.apply(mockState));
+  }
+
+  @Test public void testAddLateTriggers() throws IllegalAccessException {
+    TriggerBuilder<TestMessage, Collection<TestMessage>> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
+    builder.addLateTriggerOnSizeLimit(10000L);
+    // exam that both earlyTrigger and lateTriggers are set up
+    BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean> earlyTrigger =
+        (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
+    WindowState<Collection<TestMessage>> mockState = mock(WindowState.class);
+    when(mockState.getNumberMessages()).thenReturn(200L);
+    assertFalse(earlyTrigger.apply(null, mockState));
+    // check the late trigger
+    BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean> lateTrigger =
+        (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.lateTriggerField.get(builder);
+    assertFalse(lateTrigger.apply(null, mockState));
+    // set the number of messages to 10001 to trigger the late trigger
+    when(mockState.getNumberMessages()).thenReturn(10001L);
+    assertTrue(lateTrigger.apply(null, mockState));
+
+    builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
+    builder.addLateTrigger((m, s) -> s.getOutputValue().size() > 0);
+    // exam that both earlyTrigger and lateTriggers are set up
+    earlyTrigger = (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
+    mockState = mock(WindowState.class);
+    when(mockState.getNumberMessages()).thenReturn(200L);
+    assertFalse(earlyTrigger.apply(null, mockState));
+    // exam the lateTrigger
+    when(mockState.getOutputValue()).thenReturn(new ArrayList<>());
+    lateTrigger = (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.lateTriggerField.get(builder);
+    assertFalse(lateTrigger.apply(null, mockState));
+    List<TestMessage> mockList = mock(ArrayList.class);
+    when(mockList.size()).thenReturn(200);
+    when(mockState.getOutputValue()).thenReturn(mockList);
+    assertTrue(lateTrigger.apply(null, mockState));
+  }
+
+  @Test public void testAddTriggerUpdater() throws IllegalAccessException {
+    TriggerBuilder<TestMessage, Collection<TestMessage>> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
+    builder.onEarlyTrigger(c -> { c.clear(); return c;} );
+    List<TestMessage> collection = new ArrayList<TestMessage>() {{
+      for(int i = 0; i < 10; i++) {
+        this.add(new TestMessage(String.format("key-%d", i), "string-value", System.nanoTime()));
+      }
+    }};
+    // exam that earlyTriggerUpdater is set up
+    Function<WindowState<Collection<TestMessage>>, WindowState<Collection<TestMessage>>> earlyTriggerUpdater =
+        (Function<WindowState<Collection<TestMessage>>, WindowState<Collection<TestMessage>>>) this.earlyTriggerUpdater.get(builder);
+    WindowState<Collection<TestMessage>> mockState = mock(WindowState.class);
+    when(mockState.getOutputValue()).thenReturn(collection);
+    earlyTriggerUpdater.apply(mockState);
+    assertTrue(collection.isEmpty());
+
+    collection.add(new TestMessage("key-to-stay", "string-to-stay", System.nanoTime()));
+    collection.add(new TestMessage("key-to-remove", "string-to-remove", System.nanoTime()));
+    builder.onLateTrigger(c -> {
+      c.removeIf(t -> t.getKey().equals("key-to-remove"));
+      return c;
+    });
+    // check the late trigger updater
+    Function<WindowState<Collection<TestMessage>>, WindowState<Collection<TestMessage>>> lateTriggerUpdater =
+        (Function<WindowState<Collection<TestMessage>>, WindowState<Collection<TestMessage>>>) this.lateTriggerUpdater.get(builder);
+    when(mockState.getOutputValue()).thenReturn(collection);
+    lateTriggerUpdater.apply(mockState);
+    assertTrue(collection.size() == 1);
+    assertFalse(collection.get(0).isDelete());
+    assertEquals(collection.get(0).getKey(), "key-to-stay");
+  }
+}


[3/5] samza git commit: SAMZA-914: Creating the fluent programming APIs w/ operators

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/test/java/org/apache/samza/operators/api/TestWindows.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/api/TestWindows.java b/samza-operator/src/test/java/org/apache/samza/operators/api/TestWindows.java
new file mode 100644
index 0000000..47a37dc
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/api/TestWindows.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.api;
+
+import org.apache.samza.operators.api.Windows.Window;
+import org.apache.samza.operators.api.internal.Trigger;
+import org.apache.samza.operators.api.internal.WindowOutput;
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
+public class TestWindows {
+
+  @Test public void testSessionWindows() throws NoSuchFieldException, IllegalAccessException {
+    // test constructing the default session window
+    Window<TestMessage, String, Collection<TestMessage>, WindowOutput<String, Collection<TestMessage>>> testWnd = Windows.intoSessions(
+        TestMessage::getKey);
+    assertTrue(testWnd instanceof Windows.SessionWindow);
+    Field wndKeyFuncField = Windows.SessionWindow.class.getDeclaredField("wndKeyFunction");
+    Field aggregatorField = Windows.SessionWindow.class.getDeclaredField("aggregator");
+    wndKeyFuncField.setAccessible(true);
+    aggregatorField.setAccessible(true);
+    Function<TestMessage, String> wndKeyFunc = (Function<TestMessage, String>) wndKeyFuncField.get(testWnd);
+    assertEquals(wndKeyFunc.apply(new TestMessage("test-key", "test-value", 0)), "test-key");
+    BiFunction<TestMessage, Collection<TestMessage>, Collection<TestMessage>> aggrFunc =
+        (BiFunction<TestMessage, Collection<TestMessage>, Collection<TestMessage>>) aggregatorField.get(testWnd);
+    TestMessage mockMsg = mock(TestMessage.class);
+    Collection<TestMessage> collection = aggrFunc.apply(mockMsg, new ArrayList<>());
+    assertTrue(collection.size() == 1);
+    assertTrue(collection.contains(mockMsg));
+
+    // test constructing the session window w/ customized session info
+    Window<TestMessage, String, Collection<Character>, WindowOutput<String, Collection<Character>>> testWnd2 = Windows.intoSessions(
+        m -> String.format("key-%d", m.getTimestamp()), m -> m.getMessage().charAt(0));
+    assertTrue(testWnd2 instanceof Windows.SessionWindow);
+    wndKeyFunc = (Function<TestMessage, String>) wndKeyFuncField.get(testWnd2);
+    aggrFunc = (BiFunction<TestMessage, Collection<TestMessage>, Collection<TestMessage>>) aggregatorField.get(testWnd2);
+    assertEquals(wndKeyFunc.apply(new TestMessage("test-key", "test-value", 0)), "key-0");
+    when(mockMsg.getMessage()).thenReturn("x-001");
+    collection = aggrFunc.apply(mockMsg, new ArrayList<>());
+    assertTrue(collection.size() == 1);
+    assertTrue(collection.contains('x'));
+
+    // test constructing session window w/ a default counter
+    Window<TestMessage, String, Integer, WindowOutput<String, Integer>> testCounter = Windows.intoSessionCounter(
+        m -> String.format("key-%d", m.getTimestamp()));
+    assertTrue(testCounter instanceof Windows.SessionWindow);
+    wndKeyFunc = (Function<TestMessage, String>) wndKeyFuncField.get(testCounter);
+    BiFunction<TestMessage, Integer, Integer> counterFn = (BiFunction<TestMessage, Integer, Integer>) aggregatorField.get(testCounter);
+    when(mockMsg.getTimestamp()).thenReturn(12345L);
+    assertEquals(wndKeyFunc.apply(mockMsg), "key-12345");
+    assertEquals(counterFn.apply(mockMsg, 1), Integer.valueOf(2));
+  }
+
+  @Test public void testSetTriggers() throws NoSuchFieldException, IllegalAccessException {
+    Window<TestMessage, String, Integer, WindowOutput<String, Integer>> testCounter = Windows.intoSessionCounter(
+        m -> String.format("key-%d", m.getTimestamp()));
+    // test session window w/ a trigger
+    TriggerBuilder<TestMessage, Integer> triggerBuilder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000L);
+    testCounter.setTriggers(triggerBuilder);
+    Trigger<TestMessage, WindowState<Integer>> expectedTrigger = triggerBuilder.build();
+    Trigger<TestMessage, WindowState<Integer>> actualTrigger = Windows.getInternalWindowFn(testCounter).getTrigger();
+    // examine all trigger fields are expected
+    Field earlyTriggerField = Trigger.class.getDeclaredField("earlyTrigger");
+    Field lateTriggerField = Trigger.class.getDeclaredField("lateTrigger");
+    Field timerTriggerField = Trigger.class.getDeclaredField("timerTrigger");
+    Field earlyTriggerUpdater = Trigger.class.getDeclaredField("earlyTriggerUpdater");
+    Field lateTriggerUpdater = Trigger.class.getDeclaredField("lateTriggerUpdater");
+    earlyTriggerField.setAccessible(true);
+    lateTriggerField.setAccessible(true);
+    timerTriggerField.setAccessible(true);
+    earlyTriggerUpdater.setAccessible(true);
+    lateTriggerUpdater.setAccessible(true);
+    assertEquals(earlyTriggerField.get(expectedTrigger), earlyTriggerField.get(actualTrigger));
+    assertEquals(lateTriggerField.get(expectedTrigger), lateTriggerField.get(actualTrigger));
+    assertEquals(timerTriggerField.get(expectedTrigger), timerTriggerField.get(actualTrigger));
+    assertEquals(earlyTriggerUpdater.get(expectedTrigger), earlyTriggerUpdater.get(actualTrigger));
+    assertEquals(lateTriggerUpdater.get(expectedTrigger), lateTriggerUpdater.get(actualTrigger));
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/test/java/org/apache/samza/operators/api/data/TestIncomingSystemMessage.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/api/data/TestIncomingSystemMessage.java b/samza-operator/src/test/java/org/apache/samza/operators/api/data/TestIncomingSystemMessage.java
new file mode 100644
index 0000000..e953078
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/api/data/TestIncomingSystemMessage.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.operators.api.data;
+
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.SystemStreamPartition;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
+public class TestIncomingSystemMessage {
+
+  @Test public void testConstructor() {
+    IncomingMessageEnvelope ime = mock(IncomingMessageEnvelope.class);
+    IncomingSystemMessage ism = new IncomingSystemMessage(ime);
+
+    Object mockKey = mock(Object.class);
+    Object mockValue = mock(Object.class);
+    LongOffset testOffset = new LongOffset("12345");
+    SystemStreamPartition mockSsp = mock(SystemStreamPartition.class);
+
+    when(ime.getKey()).thenReturn(mockKey);
+    when(ime.getMessage()).thenReturn(mockValue);
+    when(ime.getSystemStreamPartition()).thenReturn(mockSsp);
+    when(ime.getOffset()).thenReturn("12345");
+
+    assertEquals(ism.getKey(), mockKey);
+    assertEquals(ism.getMessage(), mockValue);
+    assertEquals(ism.getSystemStreamPartition(), mockSsp);
+    assertEquals(ism.getOffset(), testOffset);
+    assertFalse(ism.isDelete());
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/test/java/org/apache/samza/operators/api/data/TestLongOffset.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/api/data/TestLongOffset.java b/samza-operator/src/test/java/org/apache/samza/operators/api/data/TestLongOffset.java
new file mode 100644
index 0000000..10775ec
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/api/data/TestLongOffset.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.api.data;
+
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.mock;
+
+
+public class TestLongOffset {
+
+  @Test public void testConstructor() throws Exception {
+    LongOffset o1 = new LongOffset("12345");
+    Field offsetField = LongOffset.class.getDeclaredField("offset");
+    offsetField.setAccessible(true);
+    Long x = (Long) offsetField.get(o1);
+    assertEquals(x.longValue(), 12345L);
+
+    o1 = new LongOffset("012345");
+    x = (Long) offsetField.get(o1);
+    assertEquals(x.longValue(), 12345L);
+
+    try {
+      o1 = new LongOffset("xyz");
+      fail("Constructor of LongOffset should have failed w/ mal-formatted numbers");
+    } catch (NumberFormatException nfe) {
+      // expected
+    }
+  }
+
+  @Test public void testComparator() {
+    LongOffset o1 = new LongOffset("11111");
+    Offset other = mock(Offset.class);
+    try {
+      o1.compareTo(other);
+      fail("compareTo() should have have failed when comparing to an object of a different class");
+    } catch (IllegalArgumentException iae) {
+      // expected
+    }
+
+    LongOffset o2 = new LongOffset("-10000");
+    assertEquals(o1.compareTo(o2), 1);
+    LongOffset o3 = new LongOffset("22222");
+    assertEquals(o1.compareTo(o3), -1);
+    LongOffset o4 = new LongOffset("11111");
+    assertEquals(o1.compareTo(o4), 0);
+  }
+
+  @Test public void testEquals() {
+    LongOffset o1 = new LongOffset("12345");
+    Offset other = mock(Offset.class);
+    assertFalse(o1.equals(other));
+
+    LongOffset o2 = new LongOffset("0012345");
+    assertTrue(o1.equals(o2));
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/test/java/org/apache/samza/operators/api/internal/TestTrigger.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/api/internal/TestTrigger.java b/samza-operator/src/test/java/org/apache/samza/operators/api/internal/TestTrigger.java
new file mode 100644
index 0000000..727276a
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/api/internal/TestTrigger.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.api.internal;
+
+import org.apache.samza.operators.api.WindowState;
+import org.apache.samza.operators.api.data.Message;
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertEquals;
+
+
+public class TestTrigger {
+
+  @Test public void testConstructor() throws Exception {
+    BiFunction<Message<Object, Object>, WindowState<Integer>, Boolean> earlyTrigger = (m, s) -> s.getOutputValue() > 1000;
+    BiFunction<Message<Object, Object>, WindowState<Integer>, Boolean> lateTrigger = (m, s) -> s.getOutputValue() > 1000;
+    Function<WindowState<Integer>, Boolean> timerTrigger = s -> TimeUnit.NANOSECONDS.toMillis(s.getLastMessageTimeNs()) + 50000 < System.currentTimeMillis();
+    Function<WindowState<Integer>, WindowState<Integer>> earlyTriggerUpdater = s -> { s.setOutputValue(0); return s; };
+    Function<WindowState<Integer>, WindowState<Integer>> lateTriggerUpdater = s -> { s.setOutputValue(1); return s; };
+
+    Trigger<Message<Object, Object>, WindowState<Integer>> trigger = Trigger.createTrigger(timerTrigger, earlyTrigger, lateTrigger,
+        earlyTriggerUpdater, lateTriggerUpdater);
+
+    Field earlyTriggerField = Trigger.class.getDeclaredField("earlyTrigger");
+    Field timerTriggerField = Trigger.class.getDeclaredField("timerTrigger");
+    Field lateTriggerField = Trigger.class.getDeclaredField("lateTrigger");
+    Field earlyTriggerUpdaterField = Trigger.class.getDeclaredField("earlyTriggerUpdater");
+    Field lateTriggerUpdaterField = Trigger.class.getDeclaredField("lateTriggerUpdater");
+    earlyTriggerField.setAccessible(true);
+    lateTriggerField.setAccessible(true);
+    timerTriggerField.setAccessible(true);
+    earlyTriggerUpdaterField.setAccessible(true);
+    lateTriggerUpdaterField.setAccessible(true);
+
+    assertEquals(earlyTrigger, earlyTriggerField.get(trigger));
+    assertEquals(timerTrigger, timerTriggerField.get(trigger));
+    assertEquals(lateTrigger, lateTriggerField.get(trigger));
+    assertEquals(earlyTriggerUpdater, earlyTriggerUpdaterField.get(trigger));
+    assertEquals(lateTriggerUpdater, lateTriggerUpdaterField.get(trigger));
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
new file mode 100644
index 0000000..9445f3a
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.impl;
+
+import org.apache.samza.operators.api.TestMessage;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+import org.junit.Test;
+import org.mockito.ArgumentMatcher;
+import org.reactivestreams.Subscriber;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.*;
+
+
+public class TestOperatorImpl {
+
+  TestMessage curInputMsg;
+  MessageCollector curCollector;
+  TaskCoordinator curCoordinator;
+
+  @Test public void testSubscribers() {
+    this.curInputMsg = null;
+    this.curCollector = null;
+    this.curCoordinator = null;
+    OperatorImpl<TestMessage, TestOutputMessage> opImpl = new OperatorImpl<TestMessage, TestOutputMessage>() {
+      @Override protected void onNext(TestMessage message, MessageCollector collector, TaskCoordinator coordinator) {
+        TestOperatorImpl.this.curInputMsg = message;
+        TestOperatorImpl.this.curCollector = collector;
+        TestOperatorImpl.this.curCoordinator = coordinator;
+      }
+    };
+    // verify subscribe() added the mockSub and nextProcessors() invoked the mockSub.onNext()
+    Subscriber<ProcessorContext<TestOutputMessage>> mockSub = mock(Subscriber.class);
+    opImpl.subscribe(mockSub);
+    TestOutputMessage xOutput = mock(TestOutputMessage.class);
+    MessageCollector mockCollector = mock(MessageCollector.class);
+    TaskCoordinator mockCoordinator = mock(TaskCoordinator.class);
+    opImpl.nextProcessors(xOutput, mockCollector, mockCoordinator);
+    verify(mockSub, times(1)).onNext(argThat(new ArgumentMatcher<ProcessorContext<TestOutputMessage>>() {
+      @Override public boolean matches(Object argument) {
+        ProcessorContext<TestOutputMessage> pCntx = (ProcessorContext<TestOutputMessage>) argument;
+        return pCntx.getMessage().equals(xOutput) && pCntx.getCoordinator().equals(mockCoordinator) && pCntx.getCollector().equals(mockCollector);
+      }
+    }));
+    // verify onNext() is invoked correctly
+    TestMessage mockInput = mock(TestMessage.class);
+    ProcessorContext<TestMessage> inCntx = new ProcessorContext<>(mockInput, mockCollector, mockCoordinator);
+    opImpl.onNext(inCntx);
+    assertEquals(mockInput, this.curInputMsg);
+    assertEquals(mockCollector, this.curCollector);
+    assertEquals(mockCoordinator, this.curCoordinator);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOutputMessage.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOutputMessage.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOutputMessage.java
new file mode 100644
index 0000000..4bcf767
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOutputMessage.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.impl;
+
+import org.apache.samza.operators.api.data.Message;
+
+
+class TestOutputMessage implements Message<String, Integer> {
+  private final String key;
+  private final Integer value;
+  private final long timestamp;
+
+  TestOutputMessage(String key, Integer value, long timestamp) {
+    this.key = key;
+    this.value = value;
+    this.timestamp = timestamp;
+  }
+
+  @Override public Integer getMessage() {
+    return this.value;
+  }
+
+  @Override public String getKey() {
+    return this.key;
+  }
+
+  @Override public long getTimestamp() {
+    return this.timestamp;
+  }
+}
+

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

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSimpleOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSimpleOperatorImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSimpleOperatorImpl.java
new file mode 100644
index 0000000..50154f0
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSimpleOperatorImpl.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.impl;
+
+import org.apache.samza.operators.api.TestMessage;
+import org.apache.samza.operators.api.internal.Operators.StreamOperator;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.function.Function;
+
+import static org.mockito.Mockito.*;
+
+
+public class TestSimpleOperatorImpl {
+
+  @Test public void testSimpleOperator() {
+    StreamOperator<TestMessage, TestOutputMessage> mockOp = mock(StreamOperator.class);
+    Function<TestMessage, Collection<TestOutputMessage>> txfmFn = mock(Function.class);
+    when(mockOp.getFunction()).thenReturn(txfmFn);
+
+    SimpleOperatorImpl<TestMessage, TestOutputMessage> opImpl = spy(new SimpleOperatorImpl<>(mockOp));
+    TestMessage inMsg = mock(TestMessage.class);
+    TestOutputMessage outMsg = mock(TestOutputMessage.class);
+    Collection<TestOutputMessage> mockOutputs = new ArrayList() {{
+      this.add(outMsg);
+    }};
+    when(txfmFn.apply(inMsg)).thenReturn(mockOutputs);
+    MessageCollector mockCollector = mock(MessageCollector.class);
+    TaskCoordinator mockCoordinator = mock(TaskCoordinator.class);
+    opImpl.onNext(inMsg, mockCollector, mockCoordinator);
+    verify(txfmFn, times(1)).apply(inMsg);
+    verify(opImpl, times(1)).nextProcessors(outMsg, mockCollector, mockCoordinator);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/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
new file mode 100644
index 0000000..eb8a23a
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.impl;
+
+import org.apache.samza.operators.api.MessageStream;
+import org.apache.samza.operators.api.internal.Operators.SinkOperator;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+import org.junit.Test;
+
+import static org.mockito.Mockito.*;
+
+
+public class TestSinkOperatorImpl {
+
+  @Test public void testSinkOperator() {
+    SinkOperator<TestOutputMessage> sinkOp = mock(SinkOperator.class);
+    MessageStream.VoidFunction3<TestOutputMessage, MessageCollector, TaskCoordinator> sinkFn = mock(
+        MessageStream.VoidFunction3.class);
+    when(sinkOp.getFunction()).thenReturn(sinkFn);
+    SinkOperatorImpl<TestOutputMessage> sinkImpl = new SinkOperatorImpl<>(sinkOp);
+    TestOutputMessage mockMsg = mock(TestOutputMessage.class);
+    MessageCollector mockCollector = mock(MessageCollector.class);
+    TaskCoordinator mockCoordinator = mock(TaskCoordinator.class);
+
+    sinkImpl.onNext(mockMsg, mockCollector, mockCoordinator);
+    verify(sinkFn, times(1)).apply(mockMsg, mockCollector, mockCoordinator);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStateStoreImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStateStoreImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStateStoreImpl.java
new file mode 100644
index 0000000..eb8937a
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStateStoreImpl.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.api.TestMessage;
+import org.apache.samza.operators.api.WindowState;
+import org.apache.samza.operators.api.internal.Operators.StoreFunctions;
+import org.apache.samza.storage.kv.Entry;
+import org.apache.samza.storage.kv.KeyValueStore;
+import org.apache.samza.task.TaskContext;
+import org.junit.Test;
+
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.*;
+
+
+public class TestStateStoreImpl {
+  @Test public void testStateStoreImpl() {
+    StoreFunctions<TestMessage, String, WindowState> mockStoreFunctions = mock(StoreFunctions.class);
+    // test constructor
+    StateStoreImpl<TestMessage, String, WindowState> storeImpl = new StateStoreImpl<>(mockStoreFunctions, "myStoreName");
+    TaskContext mockContext = mock(TaskContext.class);
+    KeyValueStore<String, WindowState> mockKvStore = mock(KeyValueStore.class);
+    when(mockContext.getStore("myStoreName")).thenReturn(mockKvStore);
+    // test init()
+    storeImpl.init(mockContext);
+    verify(mockContext, times(1)).getStore("myStoreName");
+    Function<TestMessage, String> wndKeyFn = mock(Function.class);
+    when(mockStoreFunctions.getStoreKeyFinder()).thenReturn(wndKeyFn);
+    TestMessage mockMsg = mock(TestMessage.class);
+    when(wndKeyFn.apply(mockMsg)).thenReturn("myKey");
+    WindowState mockState = mock(WindowState.class);
+    when(mockKvStore.get("myKey")).thenReturn(mockState);
+    // test getState()
+    Entry<String, WindowState> storeEntry = storeImpl.getState(mockMsg);
+    assertEquals(storeEntry.getKey(), "myKey");
+    assertEquals(storeEntry.getValue(), mockState);
+    verify(wndKeyFn, times(1)).apply(mockMsg);
+    verify(mockKvStore, times(1)).get("myKey");
+    Entry<String, WindowState> oldEntry = new Entry<>("myKey", mockState);
+    WindowState mockNewState = mock(WindowState.class);
+    BiFunction<TestMessage, WindowState, WindowState> mockUpdaterFn = mock(BiFunction.class);
+    when(mockStoreFunctions.getStateUpdater()).thenReturn(mockUpdaterFn);
+    when(mockUpdaterFn.apply(mockMsg, mockState)).thenReturn(mockNewState);
+    // test updateState()
+    Entry<String, WindowState> newEntry = storeImpl.updateState(mockMsg, oldEntry);
+    assertEquals(newEntry.getKey(), "myKey");
+    assertEquals(newEntry.getValue(), mockNewState);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/test/java/org/apache/samza/operators/impl/data/serializers/SqlAvroSerdeTest.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/data/serializers/SqlAvroSerdeTest.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/data/serializers/SqlAvroSerdeTest.java
new file mode 100644
index 0000000..10ee2c7
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/data/serializers/SqlAvroSerdeTest.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.impl.data.serializers;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.BinaryEncoder;
+import org.apache.avro.io.DatumWriter;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.operators.impl.data.avro.AvroData;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+public class SqlAvroSerdeTest {
+  public static final String ORDER_SCHEMA = "{\"namespace\": \"org.apache.samza.operators\",\n"+
+      " \"type\": \"record\",\n"+
+      " \"name\": \"Order\",\n"+
+      " \"fields\": [\n"+
+      "     {\"name\": \"id\", \"type\": \"int\"},\n"+
+      "     {\"name\": \"product\",  \"type\": \"string\"},\n"+
+      "     {\"name\": \"quantity\", \"type\": \"int\"}\n"+
+      " ]\n"+
+      "}";
+
+  public static Schema orderSchema = Schema.parse(ORDER_SCHEMA);
+
+  private static Serde serde = new SqlAvroSerdeFactory().getSerde("sqlAvro", sqlAvroSerdeTestConfig());
+
+  @Test
+  public void testSqlAvroSerdeDeserialization() throws IOException {
+    AvroData decodedDatum = (AvroData)serde.fromBytes(encodeMessage(sampleOrderRecord(), orderSchema));
+
+    Assert.assertTrue(decodedDatum.schema().getType() == org.apache.samza.operators.api.data.Schema.Type.STRUCT);
+    Assert.assertTrue(decodedDatum.getFieldData("id").schema().getType() == org.apache.samza.operators.api.data.Schema.Type.INTEGER);
+    Assert.assertTrue(decodedDatum.getFieldData("quantity").schema().getType() == org.apache.samza.operators.api.data.Schema.Type.INTEGER);
+    Assert.assertTrue(decodedDatum.getFieldData("product").schema().getType() == org.apache.samza.operators.api.data.Schema.Type.STRING);
+  }
+
+  @Test
+  public void testSqlAvroSerialization() throws IOException {
+    AvroData decodedDatumOriginal = (AvroData)serde.fromBytes(encodeMessage(sampleOrderRecord(), orderSchema));
+    byte[] encodedDatum = serde.toBytes(decodedDatumOriginal);
+
+    AvroData decodedDatum = (AvroData)serde.fromBytes(encodedDatum);
+
+    Assert.assertTrue(decodedDatum.schema().getType() == org.apache.samza.operators.api.data.Schema.Type.STRUCT);
+    Assert.assertTrue(decodedDatum.getFieldData("id").schema().getType() == org.apache.samza.operators.api.data.Schema.Type.INTEGER);
+    Assert.assertTrue(decodedDatum.getFieldData("quantity").schema().getType() == org.apache.samza.operators.api.data.Schema.Type.INTEGER);
+    Assert.assertTrue(decodedDatum.getFieldData("product").schema().getType() == org.apache.samza.operators.api.data.Schema.Type.STRING);
+  }
+
+  private static Config sqlAvroSerdeTestConfig(){
+    Map<String, String> config = new HashMap<String, String>();
+    config.put("serializers.sqlAvro.schema", ORDER_SCHEMA);
+
+    return new MapConfig(config);
+  }
+
+  private static byte[] encodeMessage(GenericRecord datum, Schema avroSchema) throws IOException {
+    DatumWriter<GenericRecord> writer = new GenericDatumWriter<GenericRecord>(avroSchema);
+    ByteArrayOutputStream output = new ByteArrayOutputStream();
+    BinaryEncoder encoder = new BinaryEncoder(output);
+    writer.write(datum, encoder);
+    encoder.flush();
+
+    return  output.toByteArray();
+  }
+
+  private static GenericRecord sampleOrderRecord(){
+    GenericData.Record datum = new GenericData.Record(orderSchema);
+    datum.put("id", 1);
+    datum.put("product", "paint");
+    datum.put("quantity", 3);
+
+    return datum;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/test/java/org/apache/samza/operators/impl/window/TestSessionWindowImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/window/TestSessionWindowImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/window/TestSessionWindowImpl.java
new file mode 100644
index 0000000..6947464
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/window/TestSessionWindowImpl.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.impl.window;
+
+import org.apache.samza.operators.api.MessageStream;
+import org.apache.samza.operators.api.TestMessage;
+import org.apache.samza.operators.api.WindowState;
+import org.apache.samza.operators.api.internal.Operators.StoreFunctions;
+import org.apache.samza.operators.api.internal.Operators.WindowOperator;
+import org.apache.samza.operators.api.internal.WindowOutput;
+import org.apache.samza.operators.impl.StateStoreImpl;
+import org.apache.samza.storage.kv.Entry;
+import org.apache.samza.storage.kv.KeyValueStore;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentMatcher;
+
+import java.lang.reflect.Field;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.*;
+
+
+public class TestSessionWindowImpl {
+  Field wndStoreField = null;
+  Field txfmFnField = null;
+
+  @Before public void prep() throws NoSuchFieldException {
+    wndStoreField = SessionWindowImpl.class.getDeclaredField("wndStore");
+    txfmFnField = SessionWindowImpl.class.getDeclaredField("txfmFunction");
+    wndStoreField.setAccessible(true);
+    txfmFnField.setAccessible(true);
+  }
+
+  @Test public void testConstructor() throws IllegalAccessException, NoSuchFieldException {
+    // test constructing a SessionWindowImpl w/ expected mock functions
+    MessageStream<TestMessage> mockInputStrm = mock(MessageStream.class);
+    WindowOperator<TestMessage, String, WindowState<Integer>, WindowOutput<String, Integer>> wndOp = mock(WindowOperator.class);
+    StoreFunctions<TestMessage, String, WindowState<Integer>> mockStoreFns = mock(StoreFunctions.class);
+    when(wndOp.getStoreFunctions()).thenReturn(mockStoreFns);
+    when(wndOp.getStoreName(mockInputStrm)).thenReturn("test-wnd-store");
+    BiFunction<TestMessage, Entry<String, WindowState<Integer>>, WindowOutput<String, Integer>> mockTxfmFn = mock(BiFunction.class);
+    when(wndOp.getFunction()).thenReturn(mockTxfmFn);
+    SessionWindowImpl<TestMessage, String, WindowState<Integer>, WindowOutput<String, Integer>> sessWnd = new SessionWindowImpl<>(wndOp, mockInputStrm);
+    BiFunction<TestMessage, WindowState<Integer>, WindowOutput<String, Integer>> txfmFn =
+        (BiFunction<TestMessage, WindowState<Integer>, WindowOutput<String, Integer>>) txfmFnField.get(sessWnd);
+    assertEquals(mockTxfmFn, txfmFn);
+    StateStoreImpl<TestMessage, String, WindowState<Integer>> storeImpl =
+        (StateStoreImpl<TestMessage, String, WindowState<Integer>>) wndStoreField.get(sessWnd);
+
+    // test init() and make sure the wndStore is initialized as expected
+    TestMessage mockMsg = mock(TestMessage.class);
+    TaskContext mockContext = mock(TaskContext.class);
+    KeyValueStore<String, WindowState<Integer>> mockKvStore = mock(KeyValueStore.class);
+    when(mockContext.getStore("test-wnd-store")).thenReturn(mockKvStore);
+    Function<TestMessage, String> wndKeyFn = m -> "test-msg-key";
+    when(mockStoreFns.getStoreKeyFinder()).thenReturn(wndKeyFn);
+    WindowState<Integer> mockState = mock(WindowState.class);
+    when(mockKvStore.get("test-msg-key")).thenReturn(mockState);
+    storeImpl.init(mockContext);
+    Entry<String, WindowState<Integer>> stateEntry = storeImpl.getState(mockMsg);
+    verify(mockStoreFns, times(1)).getStoreKeyFinder();
+    verify(mockKvStore, times(1)).get("test-msg-key");
+    assertEquals(stateEntry.getKey(), "test-msg-key");
+    assertEquals(stateEntry.getValue(), mockState);
+  }
+
+  @Test public void testInitAndProcess() {
+    MessageStream<TestMessage> mockInputStrm = mock(MessageStream.class);
+    WindowOperator<TestMessage, String, WindowState<Integer>, WindowOutput<String, Integer>> wndOp = mock(WindowOperator.class);
+    StoreFunctions<TestMessage, String, WindowState<Integer>> mockStoreFns = mock(StoreFunctions.class);
+    Function<TestMessage, String> wndKeyFn = m -> "test-msg-key";
+    when(mockStoreFns.getStoreKeyFinder()).thenReturn(wndKeyFn);
+    when(wndOp.getStoreFunctions()).thenReturn(mockStoreFns);
+    when(wndOp.getStoreName(mockInputStrm)).thenReturn("test-wnd-store");
+    BiFunction<TestMessage, Entry<String, WindowState<Integer>>, WindowOutput<String, Integer>> mockTxfmFn = mock(BiFunction.class);
+    when(wndOp.getFunction()).thenReturn(mockTxfmFn);
+
+    // construct and init the SessionWindowImpl object
+    SessionWindowImpl<TestMessage, String, WindowState<Integer>, WindowOutput<String, Integer>> sessWnd = new SessionWindowImpl<>(wndOp, mockInputStrm);
+    TaskContext mockContext = mock(TaskContext.class);
+    KeyValueStore<String, WindowState<Integer>> mockKvStore = mock(KeyValueStore.class);
+    when(mockContext.getStore("test-wnd-store")).thenReturn(mockKvStore);
+    sessWnd.init(mockContext);
+
+    // test onNext() method. Make sure the right methods are invoked.
+    TestMessage mockMsg = mock(TestMessage.class);
+    MessageCollector mockCollector = mock(MessageCollector.class);
+    TaskCoordinator mockCoordinator = mock(TaskCoordinator.class);
+    BiFunction<TestMessage, WindowState<Integer>, WindowState<Integer>> stateUpdaterFn = mock(BiFunction.class);
+    when(mockStoreFns.getStateUpdater()).thenReturn(stateUpdaterFn);
+    WindowState<Integer> mockNewState = mock(WindowState.class);
+    WindowState<Integer> oldState = mock(WindowState.class);
+    when(mockKvStore.get("test-msg-key")).thenReturn(oldState);
+    when(stateUpdaterFn.apply(mockMsg, oldState)).thenReturn(mockNewState);
+    sessWnd.onNext(mockMsg, mockCollector, mockCoordinator);
+    verify(mockTxfmFn, times(1)).apply(argThat(new ArgumentMatcher<TestMessage>() {
+      @Override public boolean matches(Object argument) {
+        TestMessage xIn = (TestMessage) argument;
+        return xIn.equals(mockMsg);
+      }
+    }), argThat(new ArgumentMatcher<Entry<String, WindowState<Integer>>>() {
+      @Override public boolean matches(Object argument) {
+        Entry<String, WindowState<Integer>> xIn = (Entry<String, WindowState<Integer>>) argument;
+        return xIn.getKey().equals("test-msg-key") && xIn.getValue().equals(oldState);
+      }
+    }));
+    verify(stateUpdaterFn, times(1)).apply(mockMsg, oldState);
+    verify(mockKvStore, times(1)).put("test-msg-key", mockNewState);
+  }
+}

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java b/samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java
new file mode 100644
index 0000000..11186ea
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.task;
+
+import org.apache.samza.operators.api.MessageStreams.SystemMessageStream;
+import org.apache.samza.operators.api.TriggerBuilder;
+import org.apache.samza.operators.api.Windows;
+import org.apache.samza.operators.api.data.Message;
+import org.apache.samza.operators.api.data.Offset;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.util.Collection;
+
+
+/**
+ * Example implementation of a simple user-defined tasks w/ window operators
+ *
+ */
+public class WindowOperatorTask implements StreamOperatorTask {
+  class MessageType {
+    String field1;
+    String field2;
+  }
+
+  class JsonMessage extends InputJsonSystemMessage<MessageType> {
+
+    JsonMessage(String key, MessageType data, Offset offset, long timestamp, SystemStreamPartition partition) {
+      super(key, data, offset, timestamp, partition);
+    }
+  }
+
+  @Override public void initOperators(Collection<SystemMessageStream> sources) {
+    sources.forEach(source ->
+      source.map(m1 ->
+        new JsonMessage(
+          this.myMessageKeyFunction(m1),
+          (MessageType) m1.getMessage(),
+          m1.getOffset(),
+          m1.getTimestamp(),
+          m1.getSystemStreamPartition())).
+        window(
+          Windows.<JsonMessage, String>intoSessionCounter(
+              m -> String.format("%s-%s", m.getMessage().field1, m.getMessage().field2)).
+            setTriggers(TriggerBuilder.<JsonMessage, Integer>earlyTriggerWhenExceedWndLen(100).
+              addTimeoutSinceLastMessage(30000)))
+    );
+  }
+
+  String myMessageKeyFunction(Message<Object, Object> m) {
+    return m.getKey().toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
----------------------------------------------------------------------
diff --git a/samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java b/samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
index fbb5c59..ea9ee57 100644
--- a/samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
+++ b/samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
@@ -38,7 +38,8 @@ public class TestAvroSchemaConverter {
       " ]\n" +
       "}";
 
-  public static final Schema simpleRecord = new Schema.Parser().parse(SIMPLE_RECORD_SCHEMA);
+  public static final Schema simpleRecord = Schema.parse(SIMPLE_RECORD_SCHEMA);
+
   @Test
   public void testSimpleAvroRecord(){
     RelDataTypeFactory relDataTypeFactory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/README.md
----------------------------------------------------------------------
diff --git a/samza-sql-core/README.md b/samza-sql-core/README.md
deleted file mode 100644
index 72464dc..0000000
--- a/samza-sql-core/README.md
+++ /dev/null
@@ -1,17 +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.
--->
-samza-sql is an experimental module that is under development (SAMZA-390).

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java b/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java
deleted file mode 100644
index d1b8409..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.api.data;
-
-import java.util.List;
-import java.util.Map;
-
-
-public interface Data {
-
-  Schema schema();
-
-  Object value();
-
-  int intValue();
-
-  long longValue();
-
-  float floatValue();
-
-  double doubleValue();
-
-  boolean booleanValue();
-
-  String strValue();
-
-  byte[] bytesValue();
-
-  List<Object> arrayValue();
-
-  Map<Object, Object> mapValue();
-
-  Data getElement(int index);
-
-  Data getFieldData(String fldName);
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java b/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java
deleted file mode 100644
index 80ba455..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.api.data;
-
-import java.util.HashMap;
-import java.util.Map;
-
-
-/**
- * This class defines the name scheme for the collective data entities in Samza Stream SQL, i.e. tables and streams.
- */
-public class EntityName {
-  /**
-   * {@code EntityType} defines the types of the entity names
-   *
-   */
-  private enum EntityType {
-    TABLE,
-    STREAM
-  };
-
-  /**
-   * Type of the entity name
-   */
-  private final EntityType type;
-
-  /**
-   * Formatted name of the entity.
-   *
-   * <p>This formatted name of the entity should be unique identifier for the corresponding table/stream in the system.
-   * e.g. for a Kafka system stream named "mystream", the formatted name should be "kafka:mystream".
-   */
-  private final String name;
-
-  /**
-   * Static map of already allocated table names
-   */
-  private static Map<String, EntityName> tables = new HashMap<String, EntityName>();
-
-  /**
-   * Static map of already allocated stream names
-   */
-  private static Map<String, EntityName> streams = new HashMap<String, EntityName>();
-
-  /**
-   * Private ctor to create entity names
-   *
-   * @param type Type of the entity name
-   * @param name Formatted name of the entity
-   */
-  private EntityName(EntityType type, String name) {
-    this.type = type;
-    this.name = name;
-  }
-
-  @Override
-  public String toString() {
-    return String.format("%s:%s", this.type, this.name);
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (other instanceof EntityName) {
-      EntityName otherEntity = (EntityName) other;
-      return this.type.equals(otherEntity.type) && this.name.equals(otherEntity.name);
-    }
-    return false;
-  }
-
-  /**
-   * Check to see whether this entity name is for a table
-   *
-   * @return true if the entity type is {@code EntityType.TABLE}; false otherwise
-   */
-  public boolean isTable() {
-    return this.type.equals(EntityType.TABLE);
-  }
-
-  /**
-   * Check to see whether this entity name is for a stream
-   *
-   * @return true if the entity type is {@code EntityType.STREAM}; false otherwise
-   */
-  public boolean isStream() {
-    return this.type.equals(EntityType.STREAM);
-  }
-
-  /**
-   * Get the formatted entity name
-   *
-   * @return The formatted entity name
-   */
-  public String getName() {
-    return this.name;
-  }
-
-  /**
-   * Static method to get the instance of {@code EntityName} with type {@code EntityType.TABLE}
-   *
-   * @param name The formatted entity name of the relation
-   * @return A <code>EntityName</code> for a relation
-   */
-  public static EntityName getTableName(String name) {
-    if (tables.get(name) == null) {
-      tables.put(name, new EntityName(EntityType.TABLE, name));
-    }
-    return tables.get(name);
-  }
-
-  /**
-   * Static method to get the instance of <code>EntityName</code> with type <code>EntityType.STREAM</code>
-   *
-   * @param name The formatted entity name of the stream
-   * @return A <code>EntityName</code> for a stream
-   */
-  public static EntityName getStreamName(String name) {
-    if (streams.get(name) == null) {
-      streams.put(name, new EntityName(EntityType.STREAM, name));
-    }
-    return streams.get(name);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java b/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java
deleted file mode 100644
index 72816a3..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.api.data;
-
-import org.apache.samza.storage.kv.KeyValueStore;
-
-
-/**
- * This class defines the general interface of {@code Relation}, which is defined as a map of {@link org.apache.samza.sql.api.data.Tuple}.
- *
- * <p>The interface is defined as an extension to {@link org.apache.samza.storage.kv.KeyValueStore}.
- *
- */
-
-public interface Relation<K> extends KeyValueStore<K, Tuple> {
-
-  /**
-   * Get the name of the relation
-   *
-   * @return The relation name
-   */
-  EntityName getName();
-}


[2/5] samza git commit: SAMZA-914: Creating the fluent programming APIs w/ operators

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java b/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java
deleted file mode 100644
index 1e8f192..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.api.data;
-
-import java.util.Map;
-
-
-public interface Schema {
-
-  enum Type {
-    INTEGER,
-    LONG,
-    FLOAT,
-    DOUBLE,
-    BOOLEAN,
-    STRING,
-    BYTES,
-    STRUCT,
-    ARRAY,
-    MAP
-  };
-
-  Type getType();
-
-  Schema getElementType();
-
-  Schema getValueType();
-
-  Map<String, Schema> getFields();
-
-  Schema getFieldType(String fldName);
-
-  Data read(Object object);
-
-  Data transform(Data inputData);
-
-  boolean equals(Schema other);
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java b/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java
deleted file mode 100644
index 931705e..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.api.data;
-
-import java.util.List;
-
-
-/**
- * This interface defines an ordered {@link org.apache.samza.sql.api.data.Relation}, which has an ordered key.
- *
- * <p> This is to define a stream created by CREATE STREAM statement
- *
- * @param <K> The ordered key for the {@code Stream} class
- */
-public interface Stream<K extends Comparable<?>> extends Relation<K> {
-  /**
-   * Get the list of field names used as the order keys for this stream
-   *
-   * @return The list of field names used to construct the order key for the stream
-   */
-  List<String> getOrderFields();
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.java b/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.java
deleted file mode 100644
index 7b4d984..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.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.sql.api.data;
-
-/**
- * This interface defines a non-ordered {@link org.apache.samza.sql.api.data.Relation}, which has a unique primary key
- *
- * <p> This is to define a table created by CREATE TABLE statement
- *
- * @param <K> The primary key for the {@code Table} class
- */
-public interface Table<K> extends Relation<K> {
-
-  /**
-   * Get the primary key field name for this table
-   *
-   * @return The name of the primary key field
-   */
-  String getPrimaryKeyName();
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Tuple.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Tuple.java b/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Tuple.java
deleted file mode 100644
index bea922b..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Tuple.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.api.data;
-
-import org.apache.samza.system.sql.Offset;
-
-
-/**
- * This class defines the generic interface of <code>Tuple</code>, which is a entry from the incoming stream, or one row in a <code>Relation</code>.
- *
- * <p>The <code>Tuple</code> models the basic operatible unit in streaming SQL processes in Samza.
- *
- */
-public interface Tuple {
-
-  /**
-   * Access method to get the corresponding message body in the tuple
-   *
-   * @return Message object in the tuple
-   */
-  Data getMessage();
-
-  /**
-   * Method to indicate whether the tuple is a delete tuple or an insert tuple
-   *
-   * @return A boolean value indicates whether the current tuple is a delete or insert message
-   */
-  boolean isDelete();
-
-  /**
-   * Access method to the key of the tuple
-   *
-   * @return The <code>key</code> of the tuple
-   */
-  Data getKey();
-
-  /**
-   * Get the stream name of the tuple. Note this stream name should be unique in the system.
-   *
-   * @return The stream name which this tuple belongs to
-   */
-  EntityName getEntityName();
-
-  /**
-   * Get the message creation timestamp of the tuple.
-   *
-   * @return The tuple's creation timestamp in nano seconds.
-   */
-  long getCreateTimeNano();
-
-  /**
-   * Get the offset of the tuple in the stream. This should be used to uniquely identify a tuple in a stream.
-   *
-   * @return The offset of the tuple in the stream.
-   */
-  Offset getOffset();
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/Operator.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/Operator.java b/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/Operator.java
deleted file mode 100644
index d6f6b57..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/Operator.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.api.operators;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.sql.api.data.Relation;
-import org.apache.samza.sql.api.data.Tuple;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskContext;
-import org.apache.samza.task.TaskCoordinator;
-
-
-public interface Operator {
-  /**
-   * Method to initialize the operator
-   *
-   * @param config The configuration object
-   * @param context The task context
-   * @throws Exception Throws Exception if failed to initialize the operator
-   */
-  void init(Config config, TaskContext context) throws Exception;
-
-  /**
-   * Method to perform a relational logic on the input relation
-   *
-   * <p> The actual implementation of relational logic is performed by the implementation of this method.
-   *
-   * @param deltaRelation The changed rows in the input relation, including the inserts/deletes/updates
-   * @param collector The {@link org.apache.samza.task.MessageCollector} that accepts outputs from the operator
-   * @param coordinator The {@link org.apache.samza.task.TaskCoordinator} in the context
-   * @throws Exception Throws exception if failed
-   */
-  void process(Relation deltaRelation, MessageCollector collector, TaskCoordinator coordinator)
-      throws Exception;
-
-  /**
-   * Method to process on an input tuple.
-   *
-   * @param tuple The input tuple, which has the incoming message from a stream
-   * @param collector The {@link org.apache.samza.task.MessageCollector} that accepts outputs from the operator
-   * @param coordinator The {@link org.apache.samza.task.TaskCoordinator} in the context
-   * @throws Exception Throws exception if failed
-   */
-  void process(Tuple tuple, MessageCollector collector, TaskCoordinator coordinator) throws Exception;
-
-  /**
-   * Method to refresh the result when a timer expires
-   *
-   * @param timeNano The current system time in nano second
-   * @param collector The {@link org.apache.samza.task.MessageCollector} that accepts outputs from the operator
-   * @param coordinator The {@link org.apache.samza.task.TaskCoordinator} in the context
-   * @throws Exception Throws exception if failed
-   */
-  void refresh(long timeNano, MessageCollector collector, TaskCoordinator coordinator) throws Exception;
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorCallback.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorCallback.java b/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorCallback.java
deleted file mode 100644
index fb2aa89..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorCallback.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.sql.api.operators;
-
-import org.apache.samza.sql.api.data.Relation;
-import org.apache.samza.sql.api.data.Tuple;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-
-
-/**
- * Defines the callback functions to allow customized functions to be invoked before process and before sending the result
- */
-public interface OperatorCallback {
-  /**
-   * Method to be invoked before the operator actually process the input tuple
-   *
-   * @param tuple The incoming tuple
-   * @param collector The {@link org.apache.samza.task.MessageCollector} in context
-   * @param coordinator The {@link org.apache.samza.task.TaskCoordinator} in context
-   * @return The tuple to be processed; return {@code null} if there is nothing to be processed
-   */
-  Tuple beforeProcess(Tuple tuple, MessageCollector collector, TaskCoordinator coordinator);
-
-  /**
-   * Method to be invoked before the operator actually process the input relation
-   *
-   * @param rel The input relation
-   * @param collector The {@link org.apache.samza.task.MessageCollector} in context
-   * @param coordinator The {@link org.apache.samza.task.TaskCoordinator} in context
-   * @return The relation to be processed; return {@code null} if there is nothing to be processed
-   */
-  Relation beforeProcess(Relation rel, MessageCollector collector, TaskCoordinator coordinator);
-
-  /**
-   * Method to be invoked before the operator's output tuple is sent
-   *
-   * @param tuple The output tuple
-   * @param collector The {@link org.apache.samza.task.MessageCollector} in context
-   * @param coordinator The {@link org.apache.samza.task.TaskCoordinator} in context
-   * @return The tuple to be sent; return {@code null} if there is nothing to be sent
-   */
-  Tuple afterProcess(Tuple tuple, MessageCollector collector, TaskCoordinator coordinator);
-
-  /**
-   * Method to be invoked before the operator's output relation is sent
-   *
-   * @param rel The output relation
-   * @param collector The {@link org.apache.samza.task.MessageCollector} in context
-   * @param coordinator The {@link org.apache.samza.task.TaskCoordinator} in context
-   * @return The relation to be sent; return {@code null} if there is nothing to be sent
-   */
-  Relation afterProcess(Relation rel, MessageCollector collector, TaskCoordinator coordinator);
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorRouter.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorRouter.java b/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorRouter.java
deleted file mode 100644
index 0759638..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorRouter.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.api.operators;
-
-import java.util.List;
-
-import org.apache.samza.sql.api.data.EntityName;
-
-
-/**
- * This interface class defines interface methods to connect {@link org.apache.samza.sql.api.operators.SimpleOperator}s together into a composite operator.
- *
- * <p>The {@code OperatorRouter} allows the user to attach operators to a {@link org.apache.samza.sql.api.data.Table} or
- * a {@link org.apache.samza.sql.api.data.Stream} entity, if the corresponding table/stream is included as inputs to the operator.
- * Each operator then executes its own logic and determines which table/stream to emit the output to. Through the {@code OperatorRouter},
- * the next operators attached to the corresponding output entities (i.e. table/streams) can then be invoked to continue the
- * stream process task.
- */
-public interface OperatorRouter extends Operator {
-
-  /**
-   * This method adds a {@link org.apache.samza.sql.api.operators.SimpleOperator} to the {@code OperatorRouter}.
-   *
-   * @param nextOp The {@link org.apache.samza.sql.api.operators.SimpleOperator} to be added
-   * @throws Exception Throws exception if failed
-   */
-  void addOperator(SimpleOperator nextOp) throws Exception;
-
-  /**
-   * This method gets the list of {@link org.apache.samza.sql.api.operators.SimpleOperator}s attached to an output entity (of any type)
-   *
-   * @param output The identifier of the output entity
-   * @return The list of {@link org.apache.samza.sql.api.operators.SimpleOperator} taking {@code output} as input table/stream
-   */
-  List<SimpleOperator> getNextOperators(EntityName output);
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorSpec.java b/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorSpec.java
deleted file mode 100644
index 4d670fd..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorSpec.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.api.operators;
-
-import java.util.List;
-
-import org.apache.samza.sql.api.data.EntityName;
-
-
-/**
- * This class defines a generic specification interface class for all {@link org.apache.samza.sql.api.operators.SimpleOperator}s.
- *
- * <p>The purpose of this class is to encapsulate all the details of configuration/parameters of a specific implementation of an operator.
- *
- * <p>The generic methods for an operator specification is to provide methods to get the unique ID, the list of entity names (i.e. stream name
- * in {@link org.apache.samza.sql.api.data.Table} or {@link org.apache.samza.sql.api.data.Stream} name) of input variables , and the list of entity names of the output variables.
- *
- */
-public interface OperatorSpec {
-  /**
-   * Interface method that returns the unique ID of the operator in a task
-   *
-   * @return The unique ID of the {@link org.apache.samza.sql.api.operators.SimpleOperator} object
-   */
-  String getId();
-
-  /**
-   * Access method to the list of entity names of input variables.
-   *
-   * @return A list of entity names of the inputs
-   */
-  List<EntityName> getInputNames();
-
-  /**
-   * Access method to the list of entity name of the output variable
-   *
-   * @return The entity name of the output
-   *
-   */
-  List<EntityName> getOutputNames();
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SimpleOperator.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SimpleOperator.java b/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SimpleOperator.java
deleted file mode 100644
index c49a822..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SimpleOperator.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.api.operators;
-
-
-
-/**
- * The interface for a {@code SimpleOperator} that implements a simple primitive relational logic operation
- */
-public interface SimpleOperator extends Operator {
-  /**
-   * Method to get the specification of this {@code SimpleOperator}
-   *
-   * @return The {@link org.apache.samza.sql.api.operators.OperatorSpec} object that defines the configuration/parameters of the operator
-   */
-  OperatorSpec getSpec();
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java b/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java
deleted file mode 100644
index 6f8d93b..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.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.sql.api.operators;
-
-
-
-/**
- * This class defines the interface of SQL operator factory, which creates the {@link org.apache.samza.sql.api.operators.SimpleOperator}s:
- */
-public interface SqlOperatorFactory {
-
-  /**
-   * Interface method to create/get the {@link org.apache.samza.sql.api.operators.SimpleOperator} object
-   *
-   * @param spec The specification of the {@link org.apache.samza.sql.api.operators.SimpleOperator} object
-   * @return The {@link org.apache.samza.sql.api.operators.SimpleOperator} object
-   */
-  SimpleOperator getOperator(OperatorSpec spec);
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java b/samza-sql-core/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java
deleted file mode 100644
index 72a59f2..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.sql.data;
-
-import org.apache.samza.sql.api.data.Data;
-import org.apache.samza.sql.api.data.EntityName;
-import org.apache.samza.sql.api.data.Tuple;
-import org.apache.samza.system.IncomingMessageEnvelope;
-import org.apache.samza.system.sql.LongOffset;
-import org.apache.samza.system.sql.Offset;
-
-
-/**
- * This class implements a {@link org.apache.samza.sql.api.data.Tuple} that encapsulates an {@link org.apache.samza.system.IncomingMessageEnvelope} from the system
- *
- */
-public class IncomingMessageTuple implements Tuple {
-  /**
-   * Incoming message envelope
-   */
-  private final IncomingMessageEnvelope imsg;
-
-  /**
-   * The entity name for the incoming system stream
-   */
-  private final EntityName strmEntity;
-
-  /**
-   * The receive time of this incoming message
-   */
-  private final long recvTimeNano;
-
-  /**
-   * Ctor to create a {@code IncomingMessageTuple} from {@link org.apache.samza.system.IncomingMessageEnvelope}
-   *
-   * @param imsg The incoming system message
-   */
-  public IncomingMessageTuple(IncomingMessageEnvelope imsg) {
-    this.imsg = imsg;
-    this.strmEntity =
-        EntityName.getStreamName(String.format("%s:%s", imsg.getSystemStreamPartition().getSystem(), imsg
-            .getSystemStreamPartition().getStream()));
-    this.recvTimeNano = System.nanoTime();
-  }
-
-  @Override
-  public Data getMessage() {
-    return (Data) this.imsg.getMessage();
-  }
-
-  @Override
-  public boolean isDelete() {
-    return false;
-  }
-
-  @Override
-  public Data getKey() {
-    return (Data) this.imsg.getKey();
-  }
-
-  @Override
-  public EntityName getEntityName() {
-    return this.strmEntity;
-  }
-
-  @Override
-  public long getCreateTimeNano() {
-    // TODO: this is wrong and just to keep as an placeholder. It should be replaced by the message publish time when the publish timestamp is available in the message metadata
-    return this.recvTimeNano;
-  }
-
-  @Override
-  public Offset getOffset() {
-    // TODO: need to add offset factory to generate different types of offset. This is just a placeholder,
-    // assuming incoming message carries long value as offset (i.e. Kafka case)
-    return new LongOffset(this.imsg.getOffset());
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroData.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroData.java b/samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroData.java
deleted file mode 100644
index d040be9..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroData.java
+++ /dev/null
@@ -1,262 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.data.avro;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.avro.generic.GenericArray;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.samza.sql.api.data.Data;
-import org.apache.samza.sql.api.data.Schema;
-
-
-public class AvroData implements Data {
-  protected final Object datum;
-  protected final AvroSchema schema;
-
-  private AvroData(AvroSchema schema, Object datum) {
-    this.datum = datum;
-    this.schema = schema;
-  }
-
-  @Override
-  public Schema schema() {
-    return this.schema;
-  }
-
-  @Override
-  public Object value() {
-    return this.datum;
-  }
-
-  @Override
-  public int intValue() {
-    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
-  }
-
-  @Override
-  public long longValue() {
-    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
-  }
-
-  @Override
-  public float floatValue() {
-    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
-  }
-
-  @Override
-  public double doubleValue() {
-    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
-  }
-
-  @Override
-  public boolean booleanValue() {
-    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
-  }
-
-  @Override
-  public String strValue() {
-    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
-  }
-
-  @Override
-  public byte[] bytesValue() {
-    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
-  }
-
-  @Override
-  public List<Object> arrayValue() {
-    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
-  }
-
-  @Override
-  public Map<Object, Object> mapValue() {
-    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
-  }
-
-  @Override
-  public Data getElement(int index) {
-    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
-  }
-
-  @Override
-  public Data getFieldData(String fldName) {
-    throw new UnsupportedOperationException("Can't get value for an unknown data type.");
-  }
-
-  public static AvroData getArray(AvroSchema schema, Object datum) {
-    if (schema.getType() != Schema.Type.ARRAY) {
-      throw new IllegalArgumentException("Can't create an array object with non-array schema:" + schema.getType());
-    }
-    return new AvroData(schema, datum) {
-      @SuppressWarnings("unchecked")
-      private final GenericArray<Object> array = (GenericArray<Object>) this.datum;
-
-      @Override
-      public List<Object> arrayValue() {
-        return this.array;
-      }
-
-      @Override
-      public Data getElement(int index) {
-        return this.schema.getElementType().read(array.get(index));
-      }
-
-    };
-  }
-
-  public static AvroData getMap(AvroSchema schema, Object datum) {
-    if (schema.getType() != Schema.Type.MAP) {
-      throw new IllegalArgumentException("Can't create a map object with non-map schema:" + schema.getType());
-    }
-    return new AvroData(schema, datum) {
-      @SuppressWarnings("unchecked")
-      private final Map<Object, Object> map = (Map<Object, Object>) datum;
-
-      @Override
-      public Map<Object, Object> mapValue() {
-        return this.map;
-      }
-
-      @Override
-      public Data getFieldData(String fldName) {
-        // TODO Auto-generated method stub
-        return this.schema.getValueType().read(map.get(fldName));
-      }
-
-    };
-  }
-
-  public static AvroData getStruct(AvroSchema schema, Object datum) {
-    if (schema.getType() != Schema.Type.STRUCT) {
-      throw new IllegalArgumentException("Can't create a struct object with non-struct schema:" + schema.getType());
-    }
-    return new AvroData(schema, datum) {
-      private final GenericRecord record = (GenericRecord) datum;
-
-      @Override
-      public Data getFieldData(String fldName) {
-        // TODO Auto-generated method stub
-        return this.schema.getFieldType(fldName).read(record.get(fldName));
-      }
-
-    };
-  }
-
-  public static AvroData getInt(AvroSchema schema, Object datum) {
-    if (schema.getType() != Schema.Type.INTEGER || !(datum instanceof Integer)) {
-      throw new IllegalArgumentException("data object and schema mismatch. schema:" + schema.getType() + ", data: "
-          + datum.getClass().getName());
-    }
-    return new AvroData(schema, datum) {
-      @Override
-      public int intValue() {
-        return ((Integer) datum).intValue();
-      }
-
-    };
-  }
-
-  public static AvroData getLong(AvroSchema schema, Object datum) {
-    if (schema.getType() != Schema.Type.LONG || !(datum instanceof Long)) {
-      throw new IllegalArgumentException("data object and schema mismatch. schema:" + schema.getType() + ", data: "
-          + datum.getClass().getName());
-    }
-    return new AvroData(schema, datum) {
-      @Override
-      public long longValue() {
-        return ((Long) datum).longValue();
-      }
-
-    };
-  }
-
-  public static AvroData getFloat(AvroSchema schema, Object datum) {
-    if (schema.getType() != Schema.Type.FLOAT || !(datum instanceof Float)) {
-      throw new IllegalArgumentException("data object and schema mismatch. schema:" + schema.getType() + ", data: "
-          + datum.getClass().getName());
-    }
-    return new AvroData(schema, datum) {
-      @Override
-      public float floatValue() {
-        return ((Float) datum).floatValue();
-      }
-
-    };
-  }
-
-  public static AvroData getDouble(AvroSchema schema, Object datum) {
-    if (schema.getType() != Schema.Type.DOUBLE || !(datum instanceof Double)) {
-      throw new IllegalArgumentException("data object and schema mismatch. schema:" + schema.getType() + ", data: "
-          + datum.getClass().getName());
-    }
-    return new AvroData(schema, datum) {
-      @Override
-      public double doubleValue() {
-        return ((Double) datum).doubleValue();
-      }
-
-    };
-  }
-
-  public static AvroData getBoolean(AvroSchema schema, Object datum) {
-    if (schema.getType() != Schema.Type.BOOLEAN || !(datum instanceof Boolean)) {
-      throw new IllegalArgumentException("data object and schema mismatch. schema:" + schema.getType() + ", data: "
-          + datum.getClass().getName());
-    }
-    return new AvroData(schema, datum) {
-      @Override
-      public boolean booleanValue() {
-        return ((Boolean) datum).booleanValue();
-      }
-
-    };
-  }
-
-  public static AvroData getString(AvroSchema schema, Object datum) {
-    if (schema.getType() != Schema.Type.STRING || !(datum instanceof CharSequence)) {
-      throw new IllegalArgumentException("data object and schema mismatch. schema:" + schema.getType() + ", data: "
-          + datum.getClass().getName());
-    }
-    return new AvroData(schema, datum) {
-      @Override
-      public String strValue() {
-        return ((CharSequence) datum).toString();
-      }
-
-    };
-  }
-
-  public static AvroData getBytes(AvroSchema schema, Object datum) {
-    if (schema.getType() != Schema.Type.BYTES || !(datum instanceof ByteBuffer)) {
-      throw new IllegalArgumentException("data object and schema mismatch. schema:" + schema.getType() + ", data: "
-          + datum.getClass().getName());
-    }
-    return new AvroData(schema, datum) {
-      @Override
-      public byte[] bytesValue() {
-        return ((ByteBuffer) datum).array();
-      }
-
-    };
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroSchema.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroSchema.java b/samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroSchema.java
deleted file mode 100644
index 577cf74..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroSchema.java
+++ /dev/null
@@ -1,296 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.data.avro;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.avro.Schema.Field;
-import org.apache.samza.sql.api.data.Data;
-import org.apache.samza.sql.api.data.Schema;
-
-
-public class AvroSchema implements Schema {
-
-  protected final org.apache.avro.Schema avroSchema;
-  protected final Schema.Type type;
-
-  private final static Map<org.apache.avro.Schema.Type, AvroSchema> primSchemas =
-      new HashMap<org.apache.avro.Schema.Type, AvroSchema>();
-
-  static {
-    primSchemas.put(org.apache.avro.Schema.Type.INT,
-        new AvroSchema(org.apache.avro.Schema.create(org.apache.avro.Schema.Type.INT)) {
-      @Override
-      public Data read(Object datum) {
-        return AvroData.getInt(this, datum);
-      }
-    });
-    primSchemas.put(org.apache.avro.Schema.Type.LONG,
-        new AvroSchema(org.apache.avro.Schema.create(org.apache.avro.Schema.Type.LONG)) {
-      @Override
-      public Data read(Object datum) {
-        return AvroData.getLong(this, datum);
-      }
-    });
-    primSchemas.put(org.apache.avro.Schema.Type.FLOAT,
-        new AvroSchema(org.apache.avro.Schema.create(org.apache.avro.Schema.Type.FLOAT)) {
-      @Override
-      public Data read(Object datum) {
-        return AvroData.getFloat(this, datum);
-      }
-    });
-    primSchemas.put(org.apache.avro.Schema.Type.DOUBLE,
-        new AvroSchema(org.apache.avro.Schema.create(org.apache.avro.Schema.Type.DOUBLE)) {
-      @Override
-      public Data read(Object datum) {
-        return AvroData.getDouble(this, datum);
-      }
-    });
-    primSchemas.put(org.apache.avro.Schema.Type.BOOLEAN,
-        new AvroSchema(org.apache.avro.Schema.create(org.apache.avro.Schema.Type.BOOLEAN)) {
-      @Override
-      public Data read(Object datum) {
-        return AvroData.getBoolean(this, datum);
-      }
-    });
-    primSchemas.put(org.apache.avro.Schema.Type.STRING,
-        new AvroSchema(org.apache.avro.Schema.create(org.apache.avro.Schema.Type.STRING)) {
-      @Override
-      public Data read(Object datum) {
-        return AvroData.getString(this, datum);
-      }
-    });
-    primSchemas.put(org.apache.avro.Schema.Type.BYTES,
-        new AvroSchema(org.apache.avro.Schema.create(org.apache.avro.Schema.Type.BYTES)) {
-      @Override
-      public Data read(Object datum) {
-        return AvroData.getBytes(this, datum);
-      }
-    });
-  };
-
-  public static AvroSchema getSchema(final org.apache.avro.Schema schema) {
-    Schema.Type type = mapType(schema.getType());
-    if (type != Schema.Type.ARRAY && type != Schema.Type.MAP && type != Schema.Type.STRUCT) {
-      return primSchemas.get(schema.getType());
-    }
-    // otherwise, construct the new schema
-    // TODO: It would be possible to assign each complex schema an ID and cache it w/o repeated create in-memory schema objects
-    switch (type) {
-      case ARRAY:
-        return new AvroSchema(schema) {
-          @Override
-          public Data transform(Data input) {
-            // This would get all the elements until the length of the current schema's array length
-            if (input.schema().getType() != Schema.Type.ARRAY) {
-              throw new IllegalArgumentException("Schema mismatch. Can't transfer data. input schema: "
-                  + input.schema().getType());
-            }
-            if (!input.schema().getElementType().equals(this.getElementType())) {
-              throw new IllegalArgumentException("Element schema mismatch. Can't transfer data. input schema: "
-                  + input.schema().getElementType().getType());
-            }
-            // input type matches array type
-            return AvroData.getArray(this, input.value());
-          }
-        };
-      case MAP:
-        return new AvroSchema(schema) {
-          @Override
-          public Data transform(Data input) {
-            // This would get all the elements until the length of the current schema's array length
-            if (input.schema().getType() != Schema.Type.MAP) {
-              throw new IllegalArgumentException("Schema mismatch. Can't transfer data. input schema: "
-                  + input.schema().getType());
-            }
-            if (!input.schema().getValueType().equals(this.getValueType())) {
-              throw new IllegalArgumentException("Element schema mismatch. Can't transfer data. input schema: "
-                  + input.schema().getValueType().getType());
-            }
-            // input type matches map type
-            return AvroData.getMap(this, input.value());
-          }
-        };
-      case STRUCT:
-        return new AvroSchema(schema) {
-          @SuppressWarnings("serial")
-          private final Map<String, Schema> fldSchemas = new HashMap<String, Schema>() {
-            {
-              for (Field field : schema.getFields()) {
-                put(field.name(), getSchema(field.schema()));
-              }
-            }
-          };
-
-          @Override
-          public Map<String, Schema> getFields() {
-            return this.fldSchemas;
-          }
-
-          @Override
-          public Schema getFieldType(String fldName) {
-            return this.fldSchemas.get(fldName);
-          }
-
-          @Override
-          public Data transform(Data input) {
-            // This would get all the elements until the length of the current schema's array length
-            if (input.schema().getType() != Schema.Type.STRUCT) {
-              throw new IllegalArgumentException("Schema mismatch. Can't transfer data. input schema: "
-                  + input.schema().getType());
-            }
-            // Note: this particular transform function only implements "projection to a sub-set" concept.
-            // More complex function is needed if some other concepts such as "merge from two sets of data", "allow null if does not exist" are needed
-            for (String fldName : this.fldSchemas.keySet()) {
-              // check each field schema matches input
-              Schema fldSchema = this.fldSchemas.get(fldName);
-              Schema inputFld = input.schema().getFieldType(fldName);
-              if (!fldSchema.equals(inputFld)) {
-                throw new IllegalArgumentException("Field schema mismatch. Can't transfer data for field " + fldName
-                    + ". input field schema:" + inputFld.getType() + ", this field schema: " + fldSchema.getType());
-              }
-            }
-            // input type matches struct type
-            return AvroData.getStruct(this, input.value());
-          }
-
-        };
-      default:
-        throw new IllegalArgumentException("Un-recognized complext data type:" + type);
-    }
-  }
-
-  private AvroSchema(org.apache.avro.Schema schema) {
-    this.avroSchema = schema;
-    this.type = mapType(schema.getType());
-  }
-
-  private static Type mapType(org.apache.avro.Schema.Type type) {
-    switch (type) {
-      case ARRAY:
-        return Schema.Type.ARRAY;
-      case RECORD:
-        return Schema.Type.STRUCT;
-      case MAP:
-        return Schema.Type.MAP;
-      case INT:
-        return Schema.Type.INTEGER;
-      case LONG:
-        return Schema.Type.LONG;
-      case BOOLEAN:
-        return Schema.Type.BOOLEAN;
-      case FLOAT:
-        return Schema.Type.FLOAT;
-      case DOUBLE:
-        return Schema.Type.DOUBLE;
-      case STRING:
-        return Schema.Type.STRING;
-      case BYTES:
-        return Schema.Type.BYTES;
-      default:
-        throw new IllegalArgumentException("Avro schema: " + type + " is not supported");
-    }
-  }
-
-  @Override
-  public Type getType() {
-    return this.type;
-  }
-
-  @Override
-  public Schema getElementType() {
-    if (this.type != Schema.Type.ARRAY) {
-      throw new UnsupportedOperationException("Can't getElmentType with non-array schema: " + this.type);
-    }
-    return getSchema(this.avroSchema.getElementType());
-  }
-
-  @Override
-  public Schema getValueType() {
-    if (this.type != Schema.Type.MAP) {
-      throw new UnsupportedOperationException("Can't getValueType with non-map schema: " + this.type);
-    }
-    return getSchema(this.avroSchema.getValueType());
-  }
-
-  @Override
-  public Map<String, Schema> getFields() {
-    throw new UnsupportedOperationException("Can't get field types with unknown schema type:" + this.type);
-  }
-
-  @Override
-  public Schema getFieldType(String fldName) {
-    throw new UnsupportedOperationException("Can't getFieldType with non-map/non-struct schema: " + this.type);
-  }
-
-  @Override
-  public Data read(Object object) {
-    if (this.avroSchema.getType() == org.apache.avro.Schema.Type.ARRAY) {
-      return AvroData.getArray(this, object);
-    } else if (this.avroSchema.getType() == org.apache.avro.Schema.Type.MAP) {
-      return AvroData.getMap(this, object);
-    } else if (this.avroSchema.getType() == org.apache.avro.Schema.Type.RECORD) {
-      return AvroData.getStruct(this, object);
-    }
-    throw new UnsupportedOperationException("Reading unknown complext type:" + this.type + " is not supported");
-  }
-
-  @Override
-  public Data transform(Data inputData) {
-    if (inputData.schema().getType() == Schema.Type.ARRAY || inputData.schema().getType() == Schema.Type.MAP
-        || inputData.schema().getType() == Schema.Type.STRUCT) {
-      throw new IllegalArgumentException("Complex schema should have overriden the default transform() function.");
-    }
-    if (inputData.schema().getType() != this.type) {
-      throw new IllegalArgumentException("Can't transform a mismatched primitive type. this type:" + this.type
-          + ", input type:" + inputData.schema().getType());
-    }
-    return inputData;
-  }
-
-  @Override
-  public boolean equals(Schema other) {
-    // TODO Auto-generated method stub
-    if (this.type != other.getType()) {
-      return false;
-    }
-    switch (this.type) {
-      case ARRAY:
-        // check if element types are the same
-        return this.getElementType().equals(other.getElementType());
-      case MAP:
-        // check if value types are the same
-        return this.getValueType().equals(other.getValueType());
-      case STRUCT:
-        // check if the fields schemas in this equals the other
-        // NOTE: this equals check is in consistent with the "projection to subset" concept implemented in transform()
-        for (String fieldName : this.getFields().keySet()) {
-          if (!this.getFieldType(fieldName).equals(other.getFieldType(fieldName))) {
-            return false;
-          }
-        }
-        return true;
-      default:
-        return true;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerde.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerde.java b/samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerde.java
deleted file mode 100644
index f3f7f7d..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerde.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.sql.data.serializers;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.generic.GenericDatumWriter;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.io.DatumReader;
-import org.apache.avro.io.DecoderFactory;
-import org.apache.avro.io.Encoder;
-import org.apache.avro.io.EncoderFactory;
-import org.apache.samza.SamzaException;
-import org.apache.samza.serializers.Serde;
-import org.apache.samza.sql.data.avro.AvroData;
-import org.apache.samza.sql.data.avro.AvroSchema;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-
-public class SqlAvroSerde implements Serde<AvroData> {
-  private static Logger log = LoggerFactory.getLogger(SqlAvroSerde.class);
-
-  private final Schema avroSchema;
-  private final GenericDatumReader<GenericRecord> reader;
-  private final GenericDatumWriter<Object> writer;
-
-  public SqlAvroSerde(Schema avroSchema) {
-    this.avroSchema = avroSchema;
-    this.reader = new GenericDatumReader<GenericRecord>(avroSchema);
-    this.writer = new GenericDatumWriter<Object>(avroSchema);
-  }
-
-  @Override
-  public AvroData fromBytes(byte[] bytes) {
-    GenericRecord data;
-
-    try {
-      data = reader.read(null, DecoderFactory.get().binaryDecoder(bytes, null));
-      return getAvroData(data, avroSchema);
-    } catch (IOException e) {
-      String errMsg = "Cannot decode message.";
-      log.error(errMsg, e);
-      throw new SamzaException(errMsg, e);
-    }
-  }
-
-  @Override
-  public byte[] toBytes(AvroData object) {
-    ByteArrayOutputStream out = new ByteArrayOutputStream();
-    Encoder encoder = EncoderFactory.get().binaryEncoder(out, null);
-
-    try {
-      writer.write(object.value(), encoder);
-      encoder.flush();
-      return out.toByteArray();
-    } catch (IOException e) {
-      String errMsg = "Cannot perform Avro binary encode.";
-      log.error(errMsg, e);
-      throw new SamzaException(errMsg, e);
-    }
-  }
-
-  private AvroData getAvroData(GenericRecord data, Schema type){
-    AvroSchema schema = AvroSchema.getSchema(type);
-    switch (type.getType()){
-      case RECORD:
-        return AvroData.getStruct(schema, data);
-      case ARRAY:
-        return AvroData.getArray(schema, data);
-      case MAP:
-        return AvroData.getMap(schema, data);
-      case INT:
-        return AvroData.getInt(schema, data);
-      case LONG:
-        return AvroData.getLong(schema, data);
-      case BOOLEAN:
-        return AvroData.getBoolean(schema, data);
-      case FLOAT:
-        return AvroData.getFloat(schema, data);
-      case DOUBLE:
-        return AvroData.getDouble(schema, data);
-      case STRING:
-        return AvroData.getString(schema, data);
-      case BYTES:
-        return AvroData.getBytes(schema, data);
-      default:
-        throw new IllegalArgumentException("Avro schema: " + type + " is not supported");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerdeFactory.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerdeFactory.java b/samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerdeFactory.java
deleted file mode 100644
index aad18f4..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerdeFactory.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.sql.data.serializers;
-
-import org.apache.avro.Schema;
-import org.apache.samza.SamzaException;
-import org.apache.samza.config.Config;
-import org.apache.samza.serializers.Serde;
-import org.apache.samza.serializers.SerdeFactory;
-import org.apache.samza.sql.data.avro.AvroData;
-
-public class SqlAvroSerdeFactory implements SerdeFactory<AvroData> {
-  public static final String PROP_AVRO_SCHEMA = "serializers.%s.schema";
-
-  @Override
-  public Serde<AvroData> getSerde(String name, Config config) {
-    String avroSchemaStr = config.get(String.format(PROP_AVRO_SCHEMA, name));
-    if (avroSchemaStr == null || avroSchemaStr.isEmpty()) {
-      throw new SamzaException("Cannot find avro schema for SerdeFactory '" + name + "'.");
-    }
-
-    return new SqlAvroSerde(new Schema.Parser().parse(avroSchemaStr));
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerde.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerde.java b/samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerde.java
deleted file mode 100644
index 1f0c3b2..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerde.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.data.serializers;
-
-import org.apache.samza.serializers.Serde;
-import org.apache.samza.serializers.StringSerde;
-import org.apache.samza.sql.data.string.StringData;
-
-import java.io.UnsupportedEncodingException;
-
-public class SqlStringSerde implements Serde<StringData> {
-
-    private final Serde<String> serde;
-
-    public SqlStringSerde(String encoding) {
-        this.serde = new StringSerde(encoding);
-    }
-
-    @Override
-    public StringData fromBytes(byte[] bytes) {
-          return new StringData(serde.fromBytes(bytes));
-    }
-
-    @Override
-    public byte[] toBytes(StringData object) {
-        return serde.toBytes(object.strValue());
-    }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerdeFactory.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerdeFactory.java b/samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerdeFactory.java
deleted file mode 100644
index 2564479..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerdeFactory.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.data.serializers;
-
-
-import org.apache.samza.config.Config;
-import org.apache.samza.serializers.Serde;
-import org.apache.samza.serializers.SerdeFactory;
-import org.apache.samza.sql.data.string.StringData;
-
-public class SqlStringSerdeFactory implements SerdeFactory<StringData> {
-    @Override
-    public Serde<StringData> getSerde(String name, Config config) {
-        return new SqlStringSerde(config.get("encoding", "UTF-8"));
-    }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringData.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringData.java b/samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringData.java
deleted file mode 100644
index b81d9fa..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringData.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.data.string;
-
-import org.apache.samza.sql.api.data.Data;
-import org.apache.samza.sql.api.data.Schema;
-
-import java.util.List;
-import java.util.Map;
-
-public class StringData implements Data {
-    private final Object datum;
-    private final Schema schema;
-
-    public StringData(Object datum) {
-        this.datum = datum;
-        this.schema = new StringSchema();
-    }
-
-    @Override
-    public Schema schema() {
-        return this.schema;
-    }
-
-    @Override
-    public Object value() {
-        return this.datum;
-    }
-
-    @Override
-    public int intValue() {
-        throw new UnsupportedOperationException("Can't get int value for a string type data");
-    }
-
-    @Override
-    public long longValue() {
-        throw new UnsupportedOperationException("Can't get long value for a string type data");
-    }
-
-    @Override
-    public float floatValue() {
-        throw new UnsupportedOperationException("Can't get float value for a string type data");
-    }
-
-    @Override
-    public double doubleValue() {
-        throw new UnsupportedOperationException("Can't get double value for a string type data");
-    }
-
-    @Override
-    public boolean booleanValue() {
-        throw new UnsupportedOperationException("Can't get boolean value for a string type data");
-    }
-
-    @Override
-    public String strValue() {
-        return String.valueOf(datum);
-    }
-
-    @Override
-    public byte[] bytesValue() {
-        throw new UnsupportedOperationException("Can't get bytesValue for a string type data");
-    }
-
-    @Override
-    public List<Object> arrayValue() {
-        throw new UnsupportedOperationException("Can't get arrayValue for a string type data");
-    }
-
-    @Override
-    public Map<Object, Object> mapValue() {
-        throw new UnsupportedOperationException("Can't get mapValue for a string type data");
-    }
-
-    @Override
-    public Data getElement(int index) {
-        throw new UnsupportedOperationException("Can't getElement(index) on a string type data");
-    }
-
-    @Override
-    public Data getFieldData(String fldName) {
-        throw new UnsupportedOperationException("Can't getFieldData(fieldName) for a string type data");
-    }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringSchema.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringSchema.java b/samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringSchema.java
deleted file mode 100644
index 348fc0c..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringSchema.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.data.string;
-
-import org.apache.samza.sql.api.data.Data;
-import org.apache.samza.sql.api.data.Schema;
-
-import java.util.Map;
-
-public class StringSchema implements Schema {
-    private Type type = Type.STRING;
-
-    @Override
-    public Type getType() {
-      return Type.STRING;
-    }
-
-    @Override
-    public Schema getElementType() {
-      throw new UnsupportedOperationException("Can't getElmentType with non-array schema: " + this.type);
-    }
-
-    @Override
-    public Schema getValueType() {
-        throw new UnsupportedOperationException("Can't getValueType with non-map schema: " + this.type);
-    }
-
-    @Override
-    public Map<String, Schema> getFields() {
-        throw new UnsupportedOperationException("Can't get field types with unknown schema type:" + this.type);
-    }
-
-    @Override
-    public Schema getFieldType(String fldName) {
-        throw new UnsupportedOperationException("Can't getFieldType with non-map/non-struct schema: " + this.type);
-    }
-
-    @Override
-    public Data read(Object object) {
-        return new StringData(object);
-    }
-
-    @Override
-    public Data transform(Data inputData) {
-        if (inputData.schema().getType() != this.type) {
-            throw new IllegalArgumentException("Can't transform a mismatched primitive type. this type:" + this.type
-                    + ", input type:" + inputData.schema().getType());
-        }
-        return inputData;
-    }
-
-    @Override
-    public boolean equals(Schema other) {
-        return other.getType() == this.type;
-    }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/NoopOperatorCallback.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/NoopOperatorCallback.java b/samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/NoopOperatorCallback.java
deleted file mode 100644
index c3d2266..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/NoopOperatorCallback.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.sql.operators.factory;
-
-import org.apache.samza.sql.api.data.Relation;
-import org.apache.samza.sql.api.data.Tuple;
-import org.apache.samza.sql.api.operators.OperatorCallback;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-
-
-public final class NoopOperatorCallback implements OperatorCallback {
-
-  @Override
-  public Tuple beforeProcess(Tuple tuple, MessageCollector collector, TaskCoordinator coordinator) {
-    return tuple;
-  }
-
-  @Override
-  public Relation beforeProcess(Relation rel, MessageCollector collector, TaskCoordinator coordinator) {
-    return rel;
-  }
-
-  @Override
-  public Tuple afterProcess(Tuple tuple, MessageCollector collector, TaskCoordinator coordinator) {
-    return tuple;
-  }
-
-  @Override
-  public Relation afterProcess(Relation rel, MessageCollector collector, TaskCoordinator coordinator) {
-    return rel;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java b/samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java
deleted file mode 100644
index cbc84d0..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.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.sql.operators.factory;
-
-import org.apache.samza.sql.api.operators.OperatorSpec;
-import org.apache.samza.sql.api.operators.SimpleOperator;
-import org.apache.samza.sql.api.operators.SqlOperatorFactory;
-import org.apache.samza.sql.operators.join.StreamStreamJoin;
-import org.apache.samza.sql.operators.join.StreamStreamJoinSpec;
-import org.apache.samza.sql.operators.partition.PartitionOp;
-import org.apache.samza.sql.operators.partition.PartitionSpec;
-import org.apache.samza.sql.operators.window.BoundedTimeWindow;
-import org.apache.samza.sql.operators.window.WindowSpec;
-
-
-/**
- * This simple factory class provides method to create the build-in operators per operator specification.
- * It can be extended when the build-in operators expand.
- *
- */
-public class SimpleOperatorFactoryImpl implements SqlOperatorFactory {
-
-  @Override
-  public SimpleOperator getOperator(OperatorSpec spec) {
-    if (spec instanceof PartitionSpec) {
-      return new PartitionOp((PartitionSpec) spec);
-    } else if (spec instanceof StreamStreamJoinSpec) {
-      return new StreamStreamJoin((StreamStreamJoinSpec) spec);
-    } else if (spec instanceof WindowSpec) {
-      return new BoundedTimeWindow((WindowSpec) spec);
-    }
-    throw new UnsupportedOperationException("Unsupported operator specified: " + spec.getClass().getCanonicalName());
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorImpl.java b/samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorImpl.java
deleted file mode 100644
index e66451f..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorImpl.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.operators.factory;
-
-import org.apache.samza.sql.api.data.Relation;
-import org.apache.samza.sql.api.data.Tuple;
-import org.apache.samza.sql.api.operators.OperatorCallback;
-import org.apache.samza.sql.api.operators.OperatorSpec;
-import org.apache.samza.sql.api.operators.SimpleOperator;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-import org.apache.samza.task.sql.SimpleMessageCollector;
-
-
-/**
- * An abstract class that encapsulate the basic information and methods that all operator classes should implement.
- * It implements the interface {@link org.apache.samza.sql.api.operators.SimpleOperator}
- *
- */
-public abstract class SimpleOperatorImpl implements SimpleOperator {
-  /**
-   * The specification of this operator
-   */
-  private final OperatorSpec spec;
-
-  /**
-   * The callback function
-   */
-  private final OperatorCallback callback;
-
-  /**
-   * Ctor of {@code SimpleOperatorImpl} class
-   *
-   * @param spec The specification of this operator
-   */
-  public SimpleOperatorImpl(OperatorSpec spec) {
-    this(spec, new NoopOperatorCallback());
-  }
-
-  public SimpleOperatorImpl(OperatorSpec spec, OperatorCallback callback) {
-    this.spec = spec;
-    this.callback = callback;
-  }
-
-  @Override
-  public OperatorSpec getSpec() {
-    return this.spec;
-  }
-
-  /**
-   * This method is made final s.t. the sequence of invocations between {@link org.apache.samza.sql.api.operators.OperatorCallback#beforeProcess(Relation, MessageCollector, TaskCoordinator)}
-   * and real processing of the input relation is fixed.
-   */
-  @Override
-  final public void process(Relation deltaRelation, MessageCollector collector, TaskCoordinator coordinator)
-      throws Exception {
-    Relation rel = this.callback.beforeProcess(deltaRelation, collector, coordinator);
-    if (rel == null) {
-      return;
-    }
-    this.realProcess(rel, getCollector(collector, coordinator), coordinator);
-  }
-
-  /**
-   * This method is made final s.t. the sequence of invocations between {@link org.apache.samza.sql.api.operators.OperatorCallback#beforeProcess(Tuple, MessageCollector, TaskCoordinator)}
-   * and real processing of the input tuple is fixed.
-   */
-  @Override
-  final public void process(Tuple tuple, MessageCollector collector, TaskCoordinator coordinator) throws Exception {
-    Tuple ituple = this.callback.beforeProcess(tuple, collector, coordinator);
-    if (ituple == null) {
-      return;
-    }
-    this.realProcess(ituple, getCollector(collector, coordinator), coordinator);
-  }
-
-  /**
-   * This method is made final s.t. we enforce the invocation of {@code SimpleOperatorImpl#getCollector(MessageCollector, TaskCoordinator)} before doing anything futher
-   */
-  @Override
-  final public void refresh(long timeNano, MessageCollector collector, TaskCoordinator coordinator) throws Exception {
-    this.realRefresh(timeNano, getCollector(collector, coordinator), coordinator);
-  }
-
-  private SimpleMessageCollector getCollector(MessageCollector collector, TaskCoordinator coordinator) {
-    if (!(collector instanceof SimpleMessageCollector)) {
-      return new SimpleMessageCollector(collector, coordinator, this.callback);
-    } else {
-      ((SimpleMessageCollector) collector).switchOperatorCallback(this.callback);
-      return (SimpleMessageCollector) collector;
-    }
-  }
-
-  /**
-   * Method to be overriden by each specific implementation class of operator to handle timeout event
-   *
-   * @param timeNano The time in nanosecond when the timeout event occurred
-   * @param collector The {@link org.apache.samza.task.sql.SimpleMessageCollector} in the context
-   * @param coordinator The {@link org.apache.samza.task.TaskCoordinator} in the context
-   * @throws Exception Throws exception if failed to refresh the results
-   */
-  protected abstract void realRefresh(long timeNano, SimpleMessageCollector collector, TaskCoordinator coordinator)
-      throws Exception;
-
-  /**
-   * Method to be overriden by each specific implementation class of operator to perform relational logic operation on an input {@link org.apache.samza.sql.api.data.Relation}
-   *
-   * @param rel The input relation
-   * @param collector The {@link org.apache.samza.task.sql.SimpleMessageCollector} in the context
-   * @param coordinator The {@link org.apache.samza.task.TaskCoordinator} in the context
-   * @throws Exception
-   */
-  protected abstract void realProcess(Relation rel, SimpleMessageCollector collector, TaskCoordinator coordinator)
-      throws Exception;
-
-  protected abstract void realProcess(Tuple ituple, SimpleMessageCollector collector, TaskCoordinator coordinator)
-      throws Exception;
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorSpec.java b/samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorSpec.java
deleted file mode 100644
index 56753b6..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorSpec.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.sql.operators.factory;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.samza.sql.api.data.EntityName;
-import org.apache.samza.sql.api.operators.OperatorSpec;
-
-
-/**
- * An abstract class that encapsulate the basic information and methods that all specification of operators should implement.
- * It implements {@link org.apache.samza.sql.api.operators.OperatorSpec}
- */
-public abstract class SimpleOperatorSpec implements OperatorSpec {
-  /**
-   * The identifier of the corresponding operator
-   */
-  private final String id;
-
-  /**
-   * The list of input entity names of the corresponding operator
-   */
-  private final List<EntityName> inputs = new ArrayList<EntityName>();
-
-  /**
-   * The list of output entity names of the corresponding operator
-   */
-  private final List<EntityName> outputs = new ArrayList<EntityName>();
-
-  /**
-   * Ctor of the {@code SimpleOperatorSpec} for simple {@link org.apache.samza.sql.api.operators.SimpleOperator}s w/ one input and one output
-   *
-   * @param id Unique identifier of the {@link org.apache.samza.sql.api.operators.SimpleOperator} object
-   * @param input The only input entity
-   * @param output The only output entity
-   */
-  public SimpleOperatorSpec(String id, EntityName input, EntityName output) {
-    this.id = id;
-    this.inputs.add(input);
-    this.outputs.add(output);
-  }
-
-  /**
-   * Ctor of {@code SimpleOperatorSpec} with general format: m inputs and n outputs
-   *
-   * @param id Unique identifier of the {@link org.apache.samza.sql.api.operators.SimpleOperator} object
-   * @param inputs The list of input entities
-   * @param output The list of output entities
-   */
-  public SimpleOperatorSpec(String id, List<EntityName> inputs, EntityName output) {
-    this.id = id;
-    this.inputs.addAll(inputs);
-    this.outputs.add(output);
-  }
-
-  @Override
-  public String getId() {
-    return this.id;
-  }
-
-  @Override
-  public List<EntityName> getInputNames() {
-    return this.inputs;
-  }
-
-  @Override
-  public List<EntityName> getOutputNames() {
-    return this.outputs;
-  }
-
-  /**
-   * Method to get the first output entity
-   *
-   * @return The first output entity name
-   */
-  public EntityName getOutputName() {
-    return this.outputs.get(0);
-  }
-
-  /**
-   * Method to get the first input entity
-   *
-   * @return The first input entity name
-   */
-  public EntityName getInputName() {
-    return this.inputs.get(0);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/fbdd76da/samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleRouter.java
----------------------------------------------------------------------
diff --git a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleRouter.java b/samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleRouter.java
deleted file mode 100644
index e570897..0000000
--- a/samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleRouter.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.sql.operators.factory;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.sql.api.data.EntityName;
-import org.apache.samza.sql.api.data.Relation;
-import org.apache.samza.sql.api.data.Tuple;
-import org.apache.samza.sql.api.operators.Operator;
-import org.apache.samza.sql.api.operators.OperatorRouter;
-import org.apache.samza.sql.api.operators.SimpleOperator;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskContext;
-import org.apache.samza.task.TaskCoordinator;
-import org.apache.samza.task.sql.RouterMessageCollector;
-
-
-/**
- * Example implementation of {@link org.apache.samza.sql.api.operators.OperatorRouter}
- *
- */
-public final class SimpleRouter implements OperatorRouter {
-  /**
-   * List of operators added to the {@link org.apache.samza.sql.api.operators.OperatorRouter}
-   */
-  private List<SimpleOperator> operators = new ArrayList<SimpleOperator>();
-
-  @SuppressWarnings("rawtypes")
-  /**
-   * Map of {@link org.apache.samza.sql.api.data.EntityName} to the list of operators associated with it
-   */
-  private Map<EntityName, List> nextOps = new HashMap<EntityName, List>();
-
-  /**
-   * Set of {@link org.apache.samza.sql.api.data.EntityName} as inputs to this {@code SimpleRouter}
-   */
-  private Set<EntityName> inputEntities = new HashSet<EntityName>();
-
-  /**
-   * Set of entities that are not input entities to this {@code SimpleRouter}
-   */
-  private Set<EntityName> outputEntities = new HashSet<EntityName>();
-
-  @SuppressWarnings("unchecked")
-  private void addOperator(EntityName input, SimpleOperator nextOp) {
-    if (nextOps.get(input) == null) {
-      nextOps.put(input, new ArrayList<Operator>());
-    }
-    nextOps.get(input).add(nextOp);
-    operators.add(nextOp);
-    // get the operator spec
-    for (EntityName output : nextOp.getSpec().getOutputNames()) {
-      if (inputEntities.contains(output)) {
-        inputEntities.remove(output);
-      }
-      outputEntities.add(output);
-    }
-    if (!outputEntities.contains(input)) {
-      inputEntities.add(input);
-    }
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public List<SimpleOperator> getNextOperators(EntityName entity) {
-    return nextOps.get(entity);
-  }
-
-  @Override
-  public void addOperator(SimpleOperator nextOp) {
-    List<EntityName> inputs = nextOp.getSpec().getInputNames();
-    for (EntityName input : inputs) {
-      addOperator(input, nextOp);
-    }
-  }
-
-  @Override
-  public void init(Config config, TaskContext context) throws Exception {
-    for (SimpleOperator op : this.operators) {
-      op.init(config, context);
-    }
-  }
-
-  @Override
-  public void process(Tuple ituple, MessageCollector collector, TaskCoordinator coordinator) throws Exception {
-    MessageCollector opCollector = new RouterMessageCollector(collector, coordinator, this);
-    for (Iterator<SimpleOperator> iter = this.getNextOperators(ituple.getEntityName()).iterator(); iter.hasNext();) {
-      iter.next().process(ituple, opCollector, coordinator);
-    }
-  }
-
-  @SuppressWarnings("rawtypes")
-  @Override
-  public void process(Relation deltaRelation, MessageCollector collector, TaskCoordinator coordinator) throws Exception {
-    MessageCollector opCollector = new RouterMessageCollector(collector, coordinator, this);
-    for (Iterator<SimpleOperator> iter = this.getNextOperators(deltaRelation.getName()).iterator(); iter.hasNext();) {
-      iter.next().process(deltaRelation, opCollector, coordinator);
-    }
-  }
-
-  @Override
-  public void refresh(long nanoSec, MessageCollector collector, TaskCoordinator coordinator) throws Exception {
-    MessageCollector opCollector = new RouterMessageCollector(collector, coordinator, this);
-    for (EntityName entity : inputEntities) {
-      for (Iterator<SimpleOperator> iter = this.getNextOperators(entity).iterator(); iter.hasNext();) {
-        iter.next().refresh(nanoSec, opCollector, coordinator);
-      }
-    }
-  }
-
-}